diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java index 6f6c022117c3378..654f1f96e41ee53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Analyzer.java @@ -31,6 +31,7 @@ import org.apache.doris.nereids.rules.analysis.CheckPolicy; import org.apache.doris.nereids.rules.analysis.CollectJoinConstraint; import org.apache.doris.nereids.rules.analysis.CollectSubQueryAlias; +import org.apache.doris.nereids.rules.analysis.CompressedMaterialize; import org.apache.doris.nereids.rules.analysis.EliminateDistinctConstant; import org.apache.doris.nereids.rules.analysis.EliminateGroupByConstant; import org.apache.doris.nereids.rules.analysis.EliminateLogicalSelectHint; @@ -163,6 +164,7 @@ private static List buildAnalyzerJobs(Optional topDown(new EliminateGroupByConstant()), topDown(new SimplifyAggGroupBy()), + topDown(new CompressedMaterialize()), topDown(new NormalizeAggregate()), topDown(new HavingToFilter()), bottomUp(new SemiJoinCommute()), 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 08fbc9aafec1362..f17441b9f1b884c 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 @@ -98,6 +98,8 @@ public enum RuleType { CHECK_DATA_TYPES(RuleTypeClass.CHECK), // rewrite rules + COMPRESSED_MATERIALIZE_AGG(RuleTypeClass.REWRITE), + COMPRESSED_MATERIALIZE_SORT(RuleTypeClass.REWRITE), NORMALIZE_AGGREGATE(RuleTypeClass.REWRITE), NORMALIZE_SORT(RuleTypeClass.REWRITE), NORMALIZE_REPEAT(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CompressedMaterialize.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CompressedMaterialize.java new file mode 100644 index 000000000000000..d860072ddab7893 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CompressedMaterialize.java @@ -0,0 +1,158 @@ +// 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.analysis; + +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.functions.scalar.DecodeAsVarchar; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeAsBigInt; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeAsInt; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeAsLargeInt; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeAsSmallInt; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.coercion.CharacterType; +import org.apache.doris.nereids.util.ExpressionUtils; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; +import com.google.common.collect.Maps; + +import java.util.HashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; + +/** + * select A from T group by A + * => + * select any_value(A) from T group by encode_as_int(A) + */ +public class CompressedMaterialize implements AnalysisRuleFactory { + @Override + public List buildRules() { + return ImmutableList.of( + RuleType.COMPRESSED_MATERIALIZE_AGG.build( + logicalAggregate().when(a -> ConnectContext.get() != null + && ConnectContext.get().getSessionVariable().enableCompressMaterialize) + .then(this::compressedMaterializeAggregate)), + RuleType.COMPRESSED_MATERIALIZE_SORT.build( + logicalSort().then(this::compressMaterializeSort) + ) + ); + } + + private LogicalSort compressMaterializeSort(LogicalSort sort) { + // List orderExpressions = sort.getOrderKeys().stream() + // .map(OrderKey::getExpr).collect(Collectors.toList()); + List newOrderKeys = Lists.newArrayList(); + boolean changed = false; + for (OrderKey orderKey : sort.getOrderKeys()) { + Expression expr = orderKey.getExpr(); + Optional encode = getEncodeExpression(expr); + if (encode.isPresent()) { + newOrderKeys.add(new OrderKey(encode.get(), + orderKey.isAsc(), + orderKey.isNullFirst())); + changed = true; + } else { + newOrderKeys.add(orderKey); + } + } + return changed ? sort.withOrderKeys(newOrderKeys) : sort; + } + + private Optional getEncodeExpression(Expression expression) { + DataType type = expression.getDataType(); + Expression encodeExpr = null; + if (type instanceof CharacterType) { + CharacterType ct = (CharacterType) type; + if (ct.getLen() > 1) { + // skip column from variant, like 'L.var["L_SHIPMODE"] AS TEXT' + if (ct.getLen() < 2) { + encodeExpr = new EncodeAsSmallInt(expression); + } else if (ct.getLen() < 4) { + encodeExpr = new EncodeAsInt(expression); + } else if (ct.getLen() < 7) { + encodeExpr = new EncodeAsBigInt(expression); + } else if (ct.getLen() < 15) { + encodeExpr = new EncodeAsLargeInt(expression); + } + } + } + return Optional.ofNullable(encodeExpr); + } + + /* + example: + [support] select sum(v) from t group by substring(k, 1,2) + [not support] select substring(k, 1,2), sum(v) from t group by substring(k, 1,2) + [support] select k, sum(v) from t group by k + [not support] select substring(k, 1,2), sum(v) from t group by k + [support] select A as B from T group by A + */ + private Map getEncodeGroupByExpressions(LogicalAggregate aggregate) { + Map encodeGroupbyExpressions = Maps.newHashMap(); + for (Expression gb : aggregate.getGroupByExpressions()) { + Optional encodeExpr = getEncodeExpression(gb); + encodeExpr.ifPresent(expression -> encodeGroupbyExpressions.put(gb, expression)); + } + return encodeGroupbyExpressions; + } + + private LogicalAggregate compressedMaterializeAggregate(LogicalAggregate aggregate) { + Map encodeGroupByExpressions = getEncodeGroupByExpressions(aggregate); + if (!encodeGroupByExpressions.isEmpty()) { + List newGroupByExpressions = Lists.newArrayList(); + for (Expression gp : aggregate.getGroupByExpressions()) { + newGroupByExpressions.add(encodeGroupByExpressions.getOrDefault(gp, gp)); + } + List newOutputs = Lists.newArrayList(); + Map decodeMap = new HashMap<>(); + for (Expression gp : encodeGroupByExpressions.keySet()) { + decodeMap.put(gp, new DecodeAsVarchar(encodeGroupByExpressions.get(gp))); + } + for (NamedExpression out : aggregate.getOutputExpressions()) { + Expression replaced = ExpressionUtils.replace(out, decodeMap); + if (out != replaced) { + if (out instanceof SlotReference) { + newOutputs.add(new Alias(out.getExprId(), replaced, out.getName())); + } else if (out instanceof Alias) { + newOutputs.add(((Alias) out).withChildren(replaced.children())); + } else { + // should not reach here + Preconditions.checkArgument(false, "output abnormal: " + aggregate); + } + } else { + newOutputs.add(out); + } + } + aggregate = aggregate.withGroupByAndOutput(newGroupByExpressions, newOutputs); + } + return aggregate; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java index f6f7c2d1100d0be..d40e485ca03f282 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushDownFilterThroughProject.java @@ -22,6 +22,8 @@ import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.functions.scalar.DecodeAsVarchar; +import org.apache.doris.nereids.trees.expressions.functions.scalar.EncodeStrToInteger; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; @@ -30,10 +32,12 @@ import org.apache.doris.nereids.util.PlanUtils; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import com.google.common.collect.Sets; import java.util.List; import java.util.Set; +import java.util.stream.Collectors; /** * Push down filter through project. @@ -81,7 +85,7 @@ private static Plan pushDownFilterThroughProject(LogicalFilter) project.withChildren(new LogicalFilter<>( - ExpressionUtils.replace(splitConjuncts.second, project.getAliasToProducer()), + ExpressionUtils.replace(eliminateDecodeAndEncode(splitConjuncts.second), project.getAliasToProducer()), project.child())); return PlanUtils.filterOrSelf(splitConjuncts.first, project); } @@ -99,7 +103,7 @@ private static Plan pushDownFilterThroughLimitProject( } project = project.withProjectsAndChild(project.getProjects(), new LogicalFilter<>( - ExpressionUtils.replace(splitConjuncts.second, + ExpressionUtils.replace(eliminateDecodeAndEncode(splitConjuncts.second), project.getAliasToProducer()), limit.withChildren(project.child()))); return PlanUtils.filterOrSelf(splitConjuncts.first, project); @@ -119,4 +123,31 @@ private static Pair, Set> splitConjunctsByChildOutpu } return Pair.of(remainPredicates, pushDownPredicates); } + + private static Set eliminateDecodeAndEncode(Set expressions) { + return expressions.stream() + .map(PushDownFilterThroughProject::eliminateDecodeAndEncode) + .collect(Collectors.toSet()); + } + + private static Expression eliminateDecodeAndEncode(Expression expression) { + if (expression instanceof DecodeAsVarchar && expression.child(0) instanceof EncodeStrToInteger) { + return expression.child(0).child(0); + } + boolean hasNewChild = false; + List newChildren = Lists.newArrayList(); + for (Expression child : expression.children()) { + Expression replace = eliminateDecodeAndEncode(child); + if (replace != child) { + hasNewChild = true; + newChildren.add(replace); + } else { + newChildren.add(child); + } + } + if (hasNewChild) { + return expression.withChildren(newChildren); + } + return expression; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsBigInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsBigInt.java index 59a31b4da49753c..7d798ecf3e8cab9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsBigInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsBigInt.java @@ -34,7 +34,7 @@ * ScalarFunction 'EncodeAsBigInt'. */ public class EncodeAsBigInt extends ScalarFunction - implements ExplicitlyCastableSignature, PropagateNullable { + implements ExplicitlyCastableSignature, PropagateNullable, EncodeStrToInteger { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(BigIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsInt.java index 30729354379c1ab..5c6382d6ea144da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsInt.java @@ -34,7 +34,7 @@ * ScalarFunction 'EncodeAsInt'. */ public class EncodeAsInt extends ScalarFunction - implements ExplicitlyCastableSignature, PropagateNullable { + implements ExplicitlyCastableSignature, PropagateNullable, EncodeStrToInteger { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(IntegerType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsLargeInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsLargeInt.java index 7cfce24625771f3..bb30a9a8e8aef53 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsLargeInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsLargeInt.java @@ -34,7 +34,7 @@ * ScalarFunction 'EncodeAsLargeInt'. */ public class EncodeAsLargeInt extends ScalarFunction - implements ExplicitlyCastableSignature, PropagateNullable { + implements ExplicitlyCastableSignature, PropagateNullable, EncodeStrToInteger { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(LargeIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsSmallInt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsSmallInt.java index 0809c935a575e75..355a740197c33ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsSmallInt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeAsSmallInt.java @@ -34,7 +34,7 @@ * ScalarFunction 'CompressAsSmallInt'. */ public class EncodeAsSmallInt extends ScalarFunction - implements ExplicitlyCastableSignature, PropagateNullable { + implements ExplicitlyCastableSignature, PropagateNullable, EncodeStrToInteger { public static final List SIGNATURES = ImmutableList.of( FunctionSignature.ret(SmallIntType.INSTANCE).args(VarcharType.SYSTEM_DEFAULT) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeStrToInteger.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeStrToInteger.java new file mode 100644 index 000000000000000..87a9c43687d6a3f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/EncodeStrToInteger.java @@ -0,0 +1,24 @@ +// 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.expressions.functions.scalar; + +/** + * Encode_as_XXXInt + */ +public interface EncodeStrToInteger { +} 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 404c30fe379d4a4..2a78b063a973f35 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 @@ -236,6 +236,13 @@ public int hashCode() { aggregateParam, maybeUsingStream, requireProperties); } + public PhysicalHashAggregate withGroupByExpressions(List newGroupByExpressions) { + return new PhysicalHashAggregate<>(newGroupByExpressions, outputExpressions, partitionExpressions, + aggregateParam, maybeUsingStream, groupExpression, getLogicalProperties(), + requireProperties, physicalProperties, statistics, + child()); + } + @Override public PhysicalHashAggregate withChildren(List children) { Preconditions.checkArgument(children.size() == 1); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 52ea334a14200f4..e8fb7c0af75a6f1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -2098,6 +2098,14 @@ public void setIgnoreShapePlanNodes(String ignoreShapePlanNodes) { needForward = true, fuzzy = true) public boolean enableSortSpill = false; + @VariableMgr.VarAttr( + name = "ENABLE_COMPRESS_MATERIALIZE", + description = {"控制是否启用compress materialize。默认为 true。", + "enable compress-materialize. " + + "The default value is true."}, + needForward = true, fuzzy = false) + public boolean enableCompressMaterialize = true; + @VariableMgr.VarAttr( name = ENABLE_AGG_SPILL, description = {"控制是否启用聚合算子落盘。默认为 false。", 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 a91c0dd47126fcf..e69468e9a282b1a 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 @@ -154,7 +154,8 @@ public void testCTEInHavingAndSubquery() { logicalFilter( logicalProject( logicalJoin( - logicalAggregate(), + logicalProject( + logicalAggregate()), logicalProject() ) ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out index bdf370f6b1793f2..067efb13a6557e7 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query10.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 RF5 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) ---------------------------------------PhysicalOlapScan[customer_address] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 RF5 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) +----------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] Hint log: Used: leading(customer_demographics broadcast { c broadcast ca } ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query11.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query11.out index d8cf8455c334a33..5cef5f05ad476ab 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query11.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query11.out @@ -2,52 +2,55 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1998, 1999)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1998, 1999)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF0 ss_customer_sk->[c_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +------------------PhysicalProject +--------------------filter(d_year IN (1998, 1999)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF2 ws_bill_customer_sk->[c_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] apply RFs: RF2 +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +------------------PhysicalProject +--------------------filter(d_year IN (1998, 1999)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 1999) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 1999) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 1998) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 1998) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out index 0d4af14e34d2ccb..eb614998e10a2c7 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) Hint log: Used: diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out index 212fd68ca765556..2019b4029a47cf3 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query15.out @@ -1,27 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] Hint log: Used: leading(catalog_sales date_dim { customer customer_address } ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query17.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query17.out index f2827768aa02af5..c0565fd1bf63291 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query17.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query17.out @@ -1,46 +1,47 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 -------------------------------------PhysicalProject ---------------------------------------filter((d1.d_quarter_name = '2001Q1')) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF6 -------------------------------------PhysicalProject ---------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_quarter_name = '2001Q1')) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF6 +--------------------------------------PhysicalProject +----------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] Hint log: Used: leading(catalog_sales d3 { store_sales d1 { store_returns d2 } store item } ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query18.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query18.out index e06cd4e9b57753e..a7fd2d0f92a3a8b 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query18.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query18.out @@ -1,44 +1,46 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) -------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] Hint log: Used: leading(catalog_sales cd1 { cd2 { customer customer_address } } date_dim item ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query27.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query27.out index df767a4b96e95a9..65e5b4c4d940392 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query27.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query27.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((store.s_state = 'TN')) +--------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((store.s_state = 'TN')) -----------------------------------PhysicalOlapScan[store] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] Hint log: Used: leading(store_sales customer_demographics store date_dim item ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out index f1da603468bba08..8bfe881068a78e9 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query30.out @@ -18,26 +18,27 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'AR')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'AR')) +----------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) Hint log: Used: leading(web_returns date_dim customer_address ) leading(ctr1 { customer customer_address } ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out index 4b86edab02cf417..56e026025d8f78d 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query34.out @@ -1,35 +1,36 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------PhysicalProject ---------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute[DistributionSpecHash] ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 -----------------------------------PhysicalProject -------------------------------------filter((store.s_county = 'Williamson County')) ---------------------------------------PhysicalOlapScan[store] -------------------------------PhysicalProject ---------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -------------------------------PhysicalOlapScan[household_demographics] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------PhysicalProject +----------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute[DistributionSpecHash] +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +------------------------------------PhysicalProject +--------------------------------------filter((store.s_county = 'Williamson County')) +----------------------------------------PhysicalOlapScan[store] +--------------------------------PhysicalProject +----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +--------------------------------PhysicalOlapScan[household_demographics] Hint log: Used: leading(store_sales store date_dim household_demographics ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query4.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query4.out index ee0be517b75e681..32898ecb62e3b7d 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query4.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query4.out @@ -2,73 +2,80 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF0 ss_customer_sk->[c_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() build RFs:RF2 cs_bill_customer_sk->[c_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] apply RFs: RF2 +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF4 ws_bill_customer_sk->[c_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF9 customer_id->[customer_id] ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id,customer_id,customer_id] +--------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id,customer_id,customer_id] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF8 ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF8 --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 ----------------------PhysicalProject ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out index b8219c68f1fe8b5..959f5d30c95568d 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query40.out @@ -1,32 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_item_sk->[cr_item_sk];RF3 cs_order_number->[cr_order_number] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_item_sk->[cr_item_sk];RF3 cs_order_number->[cr_order_number] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +----------------------------------PhysicalProject +------------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +--------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[warehouse] Hint log: Used: leading(catalog_returns { catalog_sales item date_dim } warehouse ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out index b65fa9047c0c87b..91df5f4ddec9093 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out index 889d5069ff97511..f420e8578c7d4bb 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query50.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query50.out index 9575ee04f0fe62f..8b32679136c1caa 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query50.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query50.out @@ -1,31 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +------------------------PhysicalOlapScan[store] Hint log: Used: leading(store_sales { store_returns d2 } broadcast d1 store ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query6.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query6.out index 43c8732d7f8553f..18dbe974af8c0c4 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query6.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query66.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query66.out index 924459179feee99..7a96568921c1605 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query66.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query69.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query69.out index 31101f12eab21ab..8ee7c8a7e0aa7ef 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query69.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) ------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 ---------------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 ---------------------------------PhysicalProject -----------------------------------filter(ca_state IN ('IL', 'ME', 'TX')) -------------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 +----------------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------filter(ca_state IN ('IL', 'ME', 'TX')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) +--------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query70.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query70.out index f01a29de46298fa..e681101eafdb7db 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query70.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query70.out @@ -13,34 +13,41 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store] Hint log: Used: leading(store_sales d1 store ) leading(store_sales date_dim store ) diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out index 473b9fded857150..d40e04f285ad81e 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_promo_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------filter(cs_bill_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_promo_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------filter(cs_bill_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out b/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out index fcbe4a8ad57c342..142ba55e2f5798e 100644 --- a/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out +++ b/regression-test/data/nereids_hint_tpcds_p0/shape/query81.out @@ -18,26 +18,27 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ctr_customer_sk->[c_customer_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'TN')) ---------------------------PhysicalOlapScan[customer_address] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ctr_customer_sk->[c_customer_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalProject +--------------------------filter((customer_address.ca_state = 'TN')) +----------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) Hint log: Used: leading(catalog_returns date_dim customer_address ) leading(customer customer_address ctr1 ) diff --git a/regression-test/data/nereids_hint_tpch_p0/shape/q12.out b/regression-test/data/nereids_hint_tpch_p0/shape/q12.out index a87109410690799..c04d5e0d01e0dc7 100644 --- a/regression-test/data/nereids_hint_tpch_p0/shape/q12.out +++ b/regression-test/data/nereids_hint_tpch_p0/shape/q12.out @@ -1,19 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] Hint log: Used: leading(orders lineitem ) diff --git a/regression-test/data/nereids_p0/compress_materialize/compress_materialize.out b/regression-test/data/nereids_p0/compress_materialize/compress_materialize.out new file mode 100644 index 000000000000000..eee04795628144d --- /dev/null +++ b/regression-test/data/nereids_p0/compress_materialize/compress_materialize.out @@ -0,0 +1,55 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !agg_exec -- +aaaaa +bbbbb + +-- !output_contains_gpk -- +aaaaa aaaaa +bbbbb bbbbb + +-- !expr -- +aaa +bbb + +-- !encodeexpr -- +12 +3 + +-- !sort -- +\N 6 + 7 +a 1 +aa 2 +b 4 +b 5 +bb 3 +中 8 +国 9 + +-- !sort -- +国 9 +中 8 +bb 3 +b 4 +b 5 +aa 2 +a 1 + 7 +\N 6 + +-- !sort -- +国 9 +中 8 +bb 3 +b 5 +b 4 +aa 2 +a 1 + 7 +\N 6 + +-- !sort -- +国 9 +中 8 +bb 3 + diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out index c1f86cac10185da..2c637ff79af5305 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.1.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((part.p_category = 'MFGR#12')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_category = 'MFGR#12')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_region = 'AMERICA')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter((supplier.s_region = 'AMERICA')) ---------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out index 5b7b82f23355a2e..a645aa975868a8b 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.2.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((supplier.s_region = 'ASIA')) +----------------------------------PhysicalOlapScan[supplier] --------------------------PhysicalProject -----------------------------filter((supplier.s_region = 'ASIA')) -------------------------------PhysicalOlapScan[supplier] +----------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) +------------------------------PhysicalOlapScan[part] ----------------------PhysicalProject -------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) ---------------------------PhysicalOlapScan[part] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out index 0523fe55e5bbc9d..35b07d5c9c95a97 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q2.3.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((part.p_brand = 'MFGR#2239')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_brand = 'MFGR#2239')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_region = 'EUROPE')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter((supplier.s_region = 'EUROPE')) ---------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out index 7d0b454caac1909..4ff658324ea03a5 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.2.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter((supplier.s_nation = 'UNITED STATES')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter((customer.c_nation = 'UNITED STATES')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter((supplier.s_nation = 'UNITED STATES')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter((customer.c_nation = 'UNITED STATES')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out index 628f3df9a5831ad..e66f2bb1a8e6c9a 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.3.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out index f725ccdbc2c1f7d..77a89293cd0d1c7 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q3.4.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_yearmonth = 'Dec1997')) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_yearmonth = 'Dec1997')) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out index efc1e0061ed88d3..669575fcff515b1 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.2.out @@ -1,32 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF2 c_custkey->[lo_custkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF1 d_datekey->[lo_orderdate] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF2 c_custkey->[lo_custkey] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF1 d_datekey->[lo_orderdate] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((supplier.s_region = 'AMERICA')) +--------------------------------------PhysicalOlapScan[supplier] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_region = 'AMERICA')) -----------------------------------PhysicalOlapScan[supplier] +--------------------------------filter(d_year IN (1997, 1998)) +----------------------------------PhysicalOlapScan[dates] --------------------------PhysicalProject -----------------------------filter(d_year IN (1997, 1998)) -------------------------------PhysicalOlapScan[dates] +----------------------------filter((customer.c_region = 'AMERICA')) +------------------------------PhysicalOlapScan[customer] ----------------------PhysicalProject -------------------------filter((customer.c_region = 'AMERICA')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter(p_mfgr IN ('MFGR#1', 'MFGR#2')) -----------------------PhysicalOlapScan[part] +------------------------filter(p_mfgr IN ('MFGR#1', 'MFGR#2')) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out index 5bd9ad4782cb3d8..4a5c29f4ca3ca77 100644 --- a/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out +++ b/regression-test/data/nereids_ssb_shape_sf100_p0/shape/q4.3.out @@ -1,31 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF3 lo_custkey->[c_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF3 lo_custkey->[c_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] apply RFs: RF3 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((supplier.s_nation = 'UNITED STATES')) +--------------------------------------PhysicalOlapScan[supplier] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_nation = 'UNITED STATES')) -----------------------------------PhysicalOlapScan[supplier] +--------------------------------filter((part.p_category = 'MFGR#14')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_category = 'MFGR#14')) -------------------------------PhysicalOlapScan[part] -----------------------PhysicalProject -------------------------filter(d_year IN (1997, 1998)) ---------------------------PhysicalOlapScan[dates] +----------------------------filter(d_year IN (1997, 1998)) +------------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out index b65fa9047c0c87b..91df5f4ddec9093 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query54.out index 397a41b34c4e60b..2c1e5cdc05399ca 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query54.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_state->[ca_state];RF5 s_county->[ca_county] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query6.out index 43c8732d7f8553f..18dbe974af8c0c4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/bs_downgrade_shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out index f68d46db52fbb83..a6e5ab7c60edd6c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/eliminate_empty/query10_empty.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) ---------------------------------------PhysicalOlapScan[customer_address] +--------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out index f68d46db52fbb83..a6e5ab7c60edd6c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) ---------------------------------------PhysicalOlapScan[customer_address] +--------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out index 35504b7f44d24e2..a6c7f181e5ad8ff 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query11.out @@ -2,53 +2,56 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1998, 1999)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1998, 1999)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------filter(d_year IN (1998, 1999)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ws_bill_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------filter(d_year IN (1998, 1999)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] ---------------PhysicalProject -----------------filter((t_w_secyear.dyear = 1999) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] ---------------------PhysicalProject -----------------------filter((t_s_secyear.dyear = 1999) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id,customer_id,customer_id] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 1998) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1998) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 +--------------------PhysicalProject +----------------------filter((t_w_firstyear.dyear = 1998) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 +----------------PhysicalProject +------------------filter((t_s_secyear.dyear = 1999) and (t_s_secyear.sale_type = 's')) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 1999) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out index 61f29b112113463..6e6c71f54a16d54 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out index 2de8417f7ab8311..f4cee0cc379f60c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query17.out index 12fa11701b619f7..34fa633896f04d4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 -------------------------------------PhysicalProject ---------------------------------------filter((d1.d_quarter_name = '2001Q1')) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF6 -------------------------------------PhysicalProject ---------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_quarter_name = '2001Q1')) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF6 +--------------------------------------PhysicalProject +----------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query18.out index ea401d9c36dc081..1b60930683fe9b6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) -------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query27.out index 47ceeb712c2a8cd..5d29068f38988cd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((store.s_state = 'TN')) +--------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((store.s_state = 'TN')) -----------------------------------PhysicalOlapScan[store] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query30.out index 8bea4f656a61328..35515fe0691a06c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'AR')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'AR')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out index 135b461a1447621..c11c22f5ba316ba 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((store.s_county = 'Williamson County')) -------------------------------------PhysicalOlapScan[store] -----------------------------PhysicalProject -------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((store.s_county = 'Williamson County')) +--------------------------------------PhysicalOlapScan[store] +------------------------------PhysicalProject +--------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out index a69bca459222b57..2d974972c455eba 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] ---------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -----------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +----------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF5 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ----------------------------------PhysicalProject ------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) --------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF5 -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out index 709da33d851bffc..331905fe3e15248 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query4.out @@ -2,74 +2,81 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] ---PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] -----------------------------PhysicalProject -------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id,customer_id,customer_id,customer_id,customer_id] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF8 customer_id->[customer_id,customer_id,customer_id] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF8 RF9 RF10 +----------------------------PhysicalProject +------------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF8 RF9 RF10 +------------------------PhysicalProject +--------------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 RF10 +--------------------PhysicalProject +----------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 RF10 +------------------PhysicalProject +--------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out index 041e5711184598a..4f8e7f620044d0b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +----------------------------------PhysicalProject +------------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +--------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out index b65fa9047c0c87b..91df5f4ddec9093 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out index 3fcfef0d8f68ba8..3de92c9c8627747 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query48.out @@ -7,23 +7,23 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('ND', 'NY', 'SD') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('GA', 'KS', 'MD') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('CO', 'MN', 'NC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF2 ss_addr_sk->[ca_address_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('ND', 'NY', 'SD') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('GA', 'KS', 'MD') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('CO', 'MN', 'NC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'GA', 'KS', 'MD', 'MN', 'NC', 'ND', 'NY', 'SD')) +--------------------PhysicalOlapScan[customer_address] apply RFs: RF2 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject --------------------------filter((store_sales.ss_net_profit <= 25000.00) and (store_sales.ss_net_profit >= 0.00) and (store_sales.ss_sales_price <= 200.00) and (store_sales.ss_sales_price >= 50.00)) -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF3 ------------------------PhysicalProject --------------------------filter(((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('2 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject -----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'GA', 'KS', 'MD', 'MN', 'NC', 'ND', 'NY', 'SD')) -------------------------PhysicalOlapScan[customer_address] -----------------PhysicalProject -------------------filter((date_dim.d_year = 2001)) ---------------------PhysicalOlapScan[date_dim] +----------------------filter((date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] ------------PhysicalProject --------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out index 889d5069ff97511..f420e8578c7d4bb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query50.out index f5c3f38463d42ca..b45ce616bee1d2a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query54.out index 397a41b34c4e60b..2c1e5cdc05399ca 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query54.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_state->[ca_state];RF5 s_county->[ca_county] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out index 43c8732d7f8553f..18dbe974af8c0c4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out index ebc2519e119218f..68a5f60cbeb86ea 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query64.out @@ -7,85 +7,86 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------PhysicalDistribute[DistributionSpecHash] ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF19 d_date_sk->[c_first_shipto_date_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[cr_item_sk,cs_item_sk,sr_item_sk,ss_item_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF18 c_customer_sk->[ss_customer_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF18 ib_income_band_sk->[hd_income_band_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF17 p_promo_sk->[ss_promo_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF17 ib_income_band_sk->[hd_income_band_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF16 ss_addr_sk->[ca_address_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF16 ca_address_sk->[c_current_addr_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF16 -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF14 ss_item_sk->[sr_item_sk];RF15 ss_ticket_number->[sr_ticket_number] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_returns] apply RFs: RF14 RF15 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF15 ca_address_sk->[ss_addr_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF13 cd_demo_sk->[ss_cdemo_sk] -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[cr_item_sk,cs_item_sk,ss_item_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF11 s_store_sk->[ss_store_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF10 ib_income_band_sk->[hd_income_band_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF9 hd_demo_sk->[ss_hdemo_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF8 cs_item_sk->[ss_item_sk] -------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF7 RF8 RF9 RF11 RF12 RF13 RF17 RF18 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter(d_year IN (1999, 2000)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------------------PhysicalProject ---------------------------------------------------------filter((sale > (2 * refund))) -----------------------------------------------------------hashAgg[GLOBAL] -------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------------------------hashAgg[LOCAL] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF14 hd_demo_sk->[c_current_hdemo_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF13 hd_demo_sk->[ss_hdemo_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF12 p_promo_sk->[ss_promo_sk] +--------------------------------------------PhysicalProject +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF11 cd_demo_sk->[c_current_cdemo_sk] +------------------------------------------------PhysicalProject +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF10 cd_demo_sk->[ss_cdemo_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[c_first_shipto_date_sk] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[c_first_sales_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF7 c_customer_sk->[ss_customer_sk] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF5 cr_item_sk->[cs_item_sk];RF6 cr_order_number->[cs_order_number] +------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF12 +----------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF4 sr_item_sk->[cr_item_sk,cs_item_sk,ss_item_sk];RF5 sr_ticket_number->[ss_ticket_number] +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF3 cs_item_sk->[ss_item_sk] +----------------------------------------------------------------------------PhysicalProject +------------------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 RF10 RF12 RF13 RF15 RF19 +--------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------filter(d_year IN (1999, 2000)) +------------------------------------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------------------------------------PhysicalProject +------------------------------------------------------------------------------filter((sale > (2 * refund))) +--------------------------------------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_item_sk->[cs_item_sk];RF1 cr_order_number->[cs_order_number] +------------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF19 +------------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF19 +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF19 --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF12 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF10 -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[income_band] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store] ---------------------------------------PhysicalProject -----------------------------------------filter((item.i_current_price <= 58.00) and (item.i_current_price >= 49.00) and i_color IN ('blush', 'lace', 'lawn', 'misty', 'orange', 'pink')) -------------------------------------------PhysicalOlapScan[item] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[promotion] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF4 c_current_addr_sk->[ca_address_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_address] apply RFs: RF4 -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF3 cd_demo_sk->[c_current_cdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[c_first_sales_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[c_current_hdemo_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 RF3 RF19 -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF0 ib_income_band_sk->[hd_income_band_sk] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF0 +----------------------------------------------------------------------PhysicalOlapScan[store] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF8 RF9 RF11 RF14 RF16 +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[promotion] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[income_band] +------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF17 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF18 --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[income_band] +--------------------PhysicalProject +----------------------PhysicalOlapScan[income_band] ----------------PhysicalProject -------------------PhysicalOlapScan[date_dim] +------------------filter((item.i_current_price <= 58.00) and (item.i_current_price >= 49.00) and i_color IN ('blush', 'lace', 'lawn', 'misty', 'orange', 'pink')) +--------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query66.out index 924459179feee99..7a96568921c1605 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query69.out index 31101f12eab21ab..8ee7c8a7e0aa7ef 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) ------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 ---------------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 ---------------------------------PhysicalProject -----------------------------------filter(ca_state IN ('IL', 'ME', 'TX')) -------------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 +----------------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------filter(ca_state IN ('IL', 'ME', 'TX')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) +--------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query70.out index ec1bdd0e99afb61..f46d7ff3d3e03a7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out index 473b9fded857150..d40e04f285ad81e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_promo_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------filter(cs_bill_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_promo_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------filter(cs_bill_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query81.out index 97ae3085a37ff79..c09a4d2af44dd25 100644 --- a/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf1000_p0/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'TN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'TN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out index bc11b07f3a3809f..f8101b31831b813 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() -------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() +--------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalProject -----------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) -------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) +--------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out index 8cab83d94f65acf..491cb062ff6d133 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query11.out @@ -2,52 +2,55 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (2001, 2002)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (2001, 2002)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out index d817c6f00537913..0fb037b0609dce4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out index 7b0fd9b10fd861c..3fbbc1ba6cd8c0c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] +----------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query17.out index c10cc616923d3c0..8608c5e3534c15a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF6 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF7 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter((d1.d_quarter_name = '2001Q1')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF6 +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF7 +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((d1.d_quarter_name = '2001Q1')) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query18.out index dcda7d5d7eb3ee2..2164dcd1cd27f3c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +--------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------PhysicalProject ---------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) -----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 +----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -----------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 1998)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query27.out index 886eca755706353..a8be055a16bb895 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out index e956468874dbb56..c9d0e3d2e35648b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'IN')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out index dad99a6b6465dd1..d9bc4269d81507c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out index 9f3bb8dab0f9e3d..09641c06af3ad0f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() -------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() +--------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out index a083e5a72ef86a7..9848fde27974006 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query4.out @@ -2,73 +2,80 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id,customer_id,customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] ----------------------PhysicalProject ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF7 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF9 ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF7 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF9 --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out index 1465471817388b4..6385a642d8f3df1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalOlapScan[warehouse] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out index c6b4daafdce0df8..845b9ab5e2e03cb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] +----------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out index dc56d7f9e81411a..3903503ed08f163 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query50.out index 8bdf05df39ed987..5ed593b117c2edc 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +--------------------------------PhysicalOlapScan[store] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query6.out index fe17bd5d3c72351..2fe5d1ab00d5fac 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[item] ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query66.out index 596cb44922b54a8..448a30647ab9671 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query69.out index e0bbeea823735d7..b1a6d0836d81fd7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() -----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() +------------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF5 ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalProject ---------------------------filter(ca_state IN ('MI', 'TX', 'VA')) -----------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalProject +----------------------------filter(ca_state IN ('MI', 'TX', 'VA')) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query70.out index 9c9d7b7638d6a5a..41c315a3884d8a6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out index 2f21640b0799298..72561c50f3d29ad 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out index 08951b2ec227719..98f023f77c1beba 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/noStatsRfPrune/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'CA')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out index 36a2c69f6b95167..352ee09144aa9a7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] -------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalProject -----------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) -------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) +--------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out index e963fc6a8caa7a3..035bb9d52145278 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query11.out @@ -2,52 +2,55 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ss_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (2001, 2002)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (2001, 2002)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out index e73d45b0732736b..30d320756649eb5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out index 1fba646d62b40ab..df3c45a2d384c3d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF1 --------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query17.out index 52da90d84ff3a84..3c56047c83427e9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF6 RF9 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF5 RF7 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter((d1.d_quarter_name = '2001Q1')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF6 RF9 +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF5 RF7 +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((d1.d_quarter_name = '2001Q1')) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query18.out index 22f67b07a4698c3..68d3302b171fe07 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +--------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------PhysicalProject ---------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) -----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -----------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 1998)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query27.out index 3eec2f7437212bd..0aac3cae957c5fd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ss_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ss_item_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out index 748165ced2fb2af..53e93243b3467c0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'IN')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out index 25429183c8fe4a9..44f18ab47b8e229 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out index f5f9508f9b6cfb5..a39e051e9dfe517 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] -------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out index 9a590246f64a4a1..13b5871c5779cf2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query4.out @@ -2,73 +2,80 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ss_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ws_bill_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id,customer_id,customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] ----------------------PhysicalProject ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF7 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF9 ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF7 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF9 --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out index aae0d7885570451..5828a951371dbef 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalOlapScan[warehouse] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out index 038ece83d7c0447..023efb28243d89e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out index dc56d7f9e81411a..3903503ed08f163 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query50.out index e7941ce875f02f7..e1fb5c9828d596d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +--------------------------------PhysicalOlapScan[store] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query54.out index 96d57c63e6f62bd..f1544cf0b3cd60e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query54.out @@ -17,7 +17,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF5 s_county->[ca_county];RF6 s_state->[ca_state] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF5 s_state->[ca_state];RF6 s_county->[ca_county] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] ----------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query6.out index 81aba1038a542ac..fc50c412111f882 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] apply RFs: RF5 ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query66.out index 7ef36371976ad65..30e394ec0060293 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query69.out index af6d7e8c85a5f6c..489c7c60182b8b7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] -----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] +------------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalProject ---------------------------filter(ca_state IN ('MI', 'TX', 'VA')) -----------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalProject +----------------------------filter(ca_state IN ('MI', 'TX', 'VA')) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query70.out index 866c026a90dd671..4614292ea19fd11 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out index 8f739a1d12b35a4..1140e5137f45fd5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out index c06a48a1f5f9228..e40e8f5035d91b5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/no_stats_shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'CA')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out index 6027b8b2684b4b7..4de40406337c3b3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ---------------------------------------PhysicalOlapScan[customer_address] +--------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out index e7ae73f8e009807..122456d14826a27 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query11.out @@ -2,53 +2,56 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter(d_year IN (2001, 2002)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +----------------------PhysicalProject +------------------------filter(d_year IN (2001, 2002)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] ---------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] ---------------------PhysicalProject -----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id,customer_id,customer_id] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 +--------------------PhysicalProject +----------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 +----------------PhysicalProject +------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out index 48ac240d961d984..17d3a28495675ee 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out index 685f61ffed3387b..3fb5053c369f630 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query17.out index 5342955a97aae22..ad361c2f1e04a7c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 -------------------------------------PhysicalProject ---------------------------------------filter((d1.d_quarter_name = '2001Q1')) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------------PhysicalProject ---------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_quarter_name = '2001Q1')) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query18.out index b1490c33c43896b..915e0e08b3d4567 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query27.out index 2dc8f171dee0d89..ade9c692f7ba354 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query30.out index 9c11294b4ec73f2..2b4f4cce8d9a4d1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------filter((date_dim.d_year = 2002)) ------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out index bb286ee190e816a..2295c980cdd3200 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalProject ---------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) -----------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +----------------------------------PhysicalOlapScan[household_demographics] +--------------------------PhysicalProject +----------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out index 49d318314dee3aa..c91693631941345 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_demographics] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out index 980ceef87cedc27..9e2ef78ab7ea161 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query4.out @@ -2,74 +2,81 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] ---PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() ------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] -----------------------------PhysicalProject -------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id,customer_id,customer_id,customer_id,customer_id] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF8 customer_id->[customer_id,customer_id,customer_id] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF8 RF9 RF10 +----------------------------PhysicalProject +------------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF8 RF9 RF10 +------------------------PhysicalProject +--------------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 RF10 +--------------------PhysicalProject +----------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 RF10 +------------------PhysicalProject +--------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out index ade38048fb9732f..a2b334f865cad43 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------------PhysicalProject +------------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +--------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out index 6f437edc5cb1a92..bd79767e40a899c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out index e34962e0847b0dd..5a3e4ad4e0d79ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query50.out index 2f0a1b10cbff1b7..5108634440efba4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 ------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query54.out index ca44d791dc42aac..3f16ecb0726d33a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query54.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_state->[ca_state];RF5 s_county->[ca_county] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out index ddb58c3887ed6fe..79795fd8b38e1c2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query66.out index 6017253a3383c5d..15f9b57a795b2d8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query69.out index a68ff0c1138094c..c78152cc022c247 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] -------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) -----------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) +------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query70.out index ae5b26647980e76..546713bec968de2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out index 668c3625c568417..5b2d657de9c18f1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF2 -----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] apply RFs: RF3 +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query81.out index 809170997c3eed0..2c5aebe5b8b4600 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'CA')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query91.out index e2a4ed89af442da..8eb26ba170c02fe 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/rf_prune/query91.out @@ -9,13 +9,13 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cr_returned_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cr_returned_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returning_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cr_returning_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 RF4 +------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 RF5 ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 c_current_addr_sk->[ca_address_sk] --------------------------------PhysicalProject @@ -34,8 +34,8 @@ PhysicalResultSink --------------------------------------filter((hd_buy_potential like '1001-5000%')) ----------------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject ---------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[call_center] --------------------PhysicalProject -----------------------PhysicalOlapScan[call_center] +----------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] 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 6027b8b2684b4b7..4de40406337c3b3 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,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ---------------------------------------PhysicalOlapScan[customer_address] +--------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out index 8abb7de87e97f9c..ff129e700362f76 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 @@ -2,53 +2,56 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------filter(d_year IN (2001, 2002)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ws_bill_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------filter(d_year IN (2001, 2002)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] ---------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] ---------------------PhysicalProject -----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id,customer_id,customer_id] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 +--------------------PhysicalProject +----------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 +----------------PhysicalProject +------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 5aad6142d951f97..50b81c67589ae3a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out index 685f61ffed3387b..3fb5053c369f630 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out index 7cc4a196c206c39..11cff6336d28e48 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,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 -------------------------------------PhysicalProject ---------------------------------------filter((d1.d_quarter_name = '2001Q1')) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 -------------------------------------PhysicalProject ---------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_quarter_name = '2001Q1')) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 +--------------------------------------PhysicalProject +----------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out index b1490c33c43896b..915e0e08b3d4567 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out index 3ddc47823586a0a..371285b5371c009 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out index b60e62f5e7bade0..2b4f4cce8d9a4d1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------filter((date_dim.d_year = 2002)) ------------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out index bb286ee190e816a..2295c980cdd3200 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalProject ---------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) -----------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +----------------------------------PhysicalOlapScan[household_demographics] +--------------------------PhysicalProject +----------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out index dcc86d240290596..d37c830e34e8120 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,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 RF4 RF5 -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_demographics] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 RF4 RF5 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] 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 709da33d851bffc..331905fe3e15248 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 @@ -2,74 +2,81 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] ---PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] -----------------------------PhysicalProject -------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id,customer_id,customer_id,customer_id,customer_id] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF8 customer_id->[customer_id,customer_id,customer_id] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF8 RF9 RF10 +----------------------------PhysicalProject +------------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF8 RF9 RF10 +------------------------PhysicalProject +--------------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 RF10 +--------------------PhysicalProject +----------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 RF10 +------------------PhysicalProject +--------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------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 5ff27658e2ed3ff..9b20475ecb8d866 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,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +----------------------------------PhysicalProject +------------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +--------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out index 6f437edc5cb1a92..bd79767e40a899c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 c_current_addr_sk->[ca_address_sk] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] apply RFs: RF0 +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out index e34962e0847b0dd..5a3e4ad4e0d79ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out index f5c3f38463d42ca..b45ce616bee1d2a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +------------------------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 c2d65d63990faef..1fb67a2b9eae2b5 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 @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_state->[ca_state];RF5 s_county->[ca_county] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] ------------------------------------------------PhysicalProject 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 43c8732d7f8553f..18dbe974af8c0c4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] 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 a4ff984e4cdc434..fedc1cffa270c1e 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 @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] 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 a68ff0c1138094c..c78152cc022c247 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,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] -------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) -----------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) +------------------------------------PhysicalOlapScan[customer_address] 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 b1074dc6ffeed5b..6f19921b730a2aa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store] 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 668c3625c568417..5b2d657de9c18f1 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,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF2 -----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] apply RFs: RF3 +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out index 27fc8430ff3e83a..4069c82aded8546 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'CA')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out index 6f3970cbb29d50e..bc2eac3de8aa780 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 @@ -9,9 +9,9 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF5 cc_call_center_sk->[cr_call_center_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cr_returned_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cr_returned_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF4 cc_call_center_sk->[cr_call_center_sk] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returning_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cr_returning_customer_sk] ----------------------------PhysicalProject @@ -34,8 +34,8 @@ PhysicalResultSink --------------------------------------filter((hd_buy_potential like '1001-5000%')) ----------------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject ---------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[call_center] --------------------PhysicalProject -----------------------PhysicalOlapScan[call_center] +----------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out index e1cace0be0ccec7..f368c522baaa211 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2000)) -------------------------------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2000)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------filter(ca_county IN ('Bonneville County', 'Boone County', 'Brown County', 'Fillmore County', 'McPherson County')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_county IN ('Bonneville County', 'Boone County', 'Brown County', 'Fillmore County', 'McPherson County')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2000)) --------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out index 9d5c47615cb77c3..98b2ab8ce7db720 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF13 ss_item_sk->[cs_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF14 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ss_item_sk,ws_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF16 ss_item_sk->[ws_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF17 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out index 31789e71f5b27aa..b2e8bff04fd8459 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF1 --------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 1998)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 1998)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query17.out index e0b281146ad099d..db86432a3cde38e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF6 RF9 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF5 RF7 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter((d1.d_quarter_name = '2000Q1')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(d_quarter_name IN ('2000Q1', '2000Q2', '2000Q3')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2000Q1', '2000Q2', '2000Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF6 RF9 +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF5 RF7 +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((d1.d_quarter_name = '2000Q1')) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(d_quarter_name IN ('2000Q1', '2000Q2', '2000Q3')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2000Q1', '2000Q2', '2000Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query18.out index 083105bf02dd546..31a5c9fa6328522 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF3 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF3 cd_demo_sk->[cs_bill_cdemo_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[c_current_cdemo_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[c_current_cdemo_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF3 RF4 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF3 RF4 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter(c_birth_month IN (1, 4, 5, 7, 8, 9)) +--------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 ------------------------------------------PhysicalProject ---------------------------------------------filter(c_birth_month IN (1, 4, 5, 7, 8, 9)) -----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 RF2 +--------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------------------filter(ca_state IN ('AL', 'AR', 'GA', 'MS', 'NC', 'TX', 'WV')) +------------------------------------------PhysicalOlapScan[customer_address] ----------------------------------PhysicalProject -------------------------------------filter(ca_state IN ('AL', 'AR', 'GA', 'MS', 'NC', 'TX', 'WV')) ---------------------------------------PhysicalOlapScan[customer_address] +------------------------------------filter((cd1.cd_education_status = 'Unknown') and (cd1.cd_gender = 'M')) +--------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------PhysicalProject ---------------------------------filter((cd1.cd_education_status = 'Unknown') and (cd1.cd_gender = 'M')) -----------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 2002)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 2002)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query27.out index f203bce45719dc1..8128f2519de7e47 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF1 cd_demo_sk->[ss_cdemo_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +------------------------------------filter((customer_demographics.cd_education_status = '2 yr Degree') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'U')) +--------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------PhysicalProject ---------------------------------filter((customer_demographics.cd_education_status = '2 yr Degree') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'U')) -----------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------filter((date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'FL', 'IN', 'NY', 'OH', 'SC')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'FL', 'IN', 'NY', 'OH', 'SC')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query30.out index cf017069f995b38..4277be3da862192 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2000)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF2 ctr_state->[ctr_state] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------PhysicalProject -----------------filter((customer_address.ca_state = 'GA')) -------------------PhysicalOlapScan[customer_address] +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF2 ctr_state->[ctr_state] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] +------------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------PhysicalProject +------------------filter((customer_address.ca_state = 'GA')) +--------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out index a883135293b7603..3a1ab4437017f3c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Arthur County', 'Halifax County', 'Lunenburg County', 'Oglethorpe County', 'Perry County', 'Salem County', 'Sumner County', 'Terrell County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('>10000', 'Unknown')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Arthur County', 'Halifax County', 'Lunenburg County', 'Oglethorpe County', 'Perry County', 'Salem County', 'Sumner County', 'Terrell County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('>10000', 'Unknown')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out index b14e619cc70b343..a373d49a63c4060 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out index 414f6e3e0bd2ed6..4e7999587f40e01 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[LEFT_OUTER_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[LEFT_OUTER_JOIN shuffle] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalOlapScan[warehouse] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2000-04-17') and (date_dim.d_date >= '2000-02-17')) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_date <= '2000-04-17') and (date_dim.d_date >= '2000-02-17')) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out index 98618affdbd763e..ca4d826194ef6fe 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +----------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 1998)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 1998)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out index 398dcef280ccea0..0ec4871b6d92323 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 2000)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query50.out index d2eda4d35debce1..b3f5ad92f12a2a8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +--------------------------------PhysicalOlapScan[store] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 9) and (d2.d_year = 1998)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((d2.d_moy = 9) and (d2.d_year = 1998)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query54.out index 4ee5ff3b075462f..140140f382f3e9b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query54.out @@ -19,7 +19,7 @@ PhysicalResultSink --------------------------------PhysicalProject ----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF5 s_county->[ca_county];RF6 s_state->[ca_state] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF5 s_state->[ca_state];RF6 s_county->[ca_county] ----------------------------------------PhysicalProject ------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] --------------------------------------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query6.out index 0c390df2e8e5f88..4f8f9a3dbc3e9ef 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] apply RFs: RF5 ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query66.out index e9d1404570f6ed5..6d97baae418cf79 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((time_dim.t_time <= 38253) and (time_dim.t_time >= 9453)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GERMA', 'MSC')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((time_dim.t_time <= 38253) and (time_dim.t_time >= 9453)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GERMA', 'MSC')) -----------------------------------PhysicalOlapScan[ship_mode] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((time_dim.t_time <= 38253) and (time_dim.t_time >= 9453)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GERMA', 'MSC')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((time_dim.t_time <= 38253) and (time_dim.t_time >= 9453)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GERMA', 'MSC')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query69.out index 115564d2edf45e6..1db1fc3c4018aa4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] -----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +------------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 2) and (date_dim.d_year = 2003)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 2) and (date_dim.d_year = 2003)) ----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 2) and (date_dim.d_year = 2003)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(ca_state IN ('AZ', 'MN', 'MO')) ---------------------------------PhysicalOlapScan[customer_address] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_demographics] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 2) and (date_dim.d_year = 2003)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------------filter(ca_state IN ('AZ', 'MN', 'MO')) +----------------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_demographics] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 2) and (date_dim.d_year = 2003)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query70.out index 3038640513b88d6..24ea74e4aebc657 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1229) and (d1.d_month_seq >= 1218)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1229) and (date_dim.d_month_seq >= 1218)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1229) and (d1.d_month_seq >= 1218)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1229) and (date_dim.d_month_seq >= 1218)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out index 47d1baebaf8c153..ae63f44c6c23713 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] -----------------------------PhysicalProject -------------------------------filter(ws_ship_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------filter(cs_ship_mode_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ws_ship_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------filter(cs_ship_mode_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query81.out index da27f4206dda1ab..93d10836ae12e62 100644 --- a/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf10t_orc/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 1998)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF3 ctr_state->[ctr_state] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF4 -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecHash] -----------------------hashAgg[LOCAL] -------------------------PhysicalDistribute[DistributionSpecExecutionAny] ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalProject -----------------filter((customer_address.ca_state = 'TX')) -------------------PhysicalOlapScan[customer_address] +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF3 ctr_state->[ctr_state] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF3 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF4 +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute[DistributionSpecHash] +------------------------hashAgg[LOCAL] +--------------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalProject +------------------filter((customer_address.ca_state = 'TX')) +--------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q12.out index 8df830dd428e58a..99ea02d4359a948 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q16.out index 7b04caaf3e087a0..95c7b4b338279ca 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q16.out @@ -1,21 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] -------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -----------------------PhysicalOlapScan[part] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +----------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +------------------------PhysicalProject +--------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +------------------------PhysicalProject +--------------------------filter((s_comment like '%Customer%Complaints%')) +----------------------------PhysicalOlapScan[supplier] +----------------------PhysicalProject +------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out index 176b64caf7833f1..498641ff0de0184 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/nostats_rf_prune/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q12.out index 8df830dd428e58a..99ea02d4359a948 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out index f04b0bc766338b7..3158a8c782c15f0 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q16.out @@ -1,21 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() -----------------PhysicalProject -------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -------------------------PhysicalOlapScan[part] -----------------PhysicalProject -------------------filter((s_comment like '%Customer%Complaints%')) ---------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +------------------------------PhysicalOlapScan[part] +----------------------PhysicalProject +------------------------filter((s_comment like '%Customer%Complaints%')) +--------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out index 207538326c7118c..99242aaac78d551 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/rf_prune/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() +--------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) -----------------------------PhysicalOlapScan[customer] +------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[orders] +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +--------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out index 8df830dd428e58a..99ea02d4359a948 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out index f04b0bc766338b7..3158a8c782c15f0 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out @@ -1,21 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() -----------------PhysicalProject -------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -------------------------PhysicalOlapScan[part] -----------------PhysicalProject -------------------filter((s_comment like '%Customer%Complaints%')) ---------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +------------------------------PhysicalOlapScan[part] +----------------------PhysicalProject +------------------------filter((s_comment like '%Customer%Complaints%')) +--------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out index 207538326c7118c..99242aaac78d551 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() +--------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) -----------------------------PhysicalOlapScan[customer] +------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[orders] +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +--------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q12.out index 8df830dd428e58a..99ea02d4359a948 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out index 7b04caaf3e087a0..95c7b4b338279ca 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q16.out @@ -1,21 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] -------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -----------------------PhysicalOlapScan[part] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +----------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +------------------------PhysicalProject +--------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +------------------------PhysicalProject +--------------------------filter((s_comment like '%Customer%Complaints%')) +----------------------------PhysicalOlapScan[supplier] +----------------------PhysicalProject +------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out index 176b64caf7833f1..498641ff0de0184 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape_no_stats/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/hint_tpch/shape/q12.out b/regression-test/data/new_shapes_p0/hint_tpch/shape/q12.out index a87109410690799..c04d5e0d01e0dc7 100644 --- a/regression-test/data/new_shapes_p0/hint_tpch/shape/q12.out +++ b/regression-test/data/new_shapes_p0/hint_tpch/shape/q12.out @@ -1,19 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] Hint log: Used: leading(orders lineitem ) diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.1.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.1.out index c1f86cac10185da..2c637ff79af5305 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.1.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.1.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((part.p_category = 'MFGR#12')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_category = 'MFGR#12')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_region = 'AMERICA')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter((supplier.s_region = 'AMERICA')) ---------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.2.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.2.out index 5b7b82f23355a2e..a645aa975868a8b 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.2.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.2.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((supplier.s_region = 'ASIA')) +----------------------------------PhysicalOlapScan[supplier] --------------------------PhysicalProject -----------------------------filter((supplier.s_region = 'ASIA')) -------------------------------PhysicalOlapScan[supplier] +----------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) +------------------------------PhysicalOlapScan[part] ----------------------PhysicalProject -------------------------filter((part.p_brand <= 'MFGR#2228') and (part.p_brand >= 'MFGR#2221')) ---------------------------PhysicalOlapScan[part] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.3.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.3.out index 0523fe55e5bbc9d..35b07d5c9c95a97 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.3.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q2.3.out @@ -1,26 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF1 s_suppkey->[lo_suppkey] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF0 p_partkey->[lo_partkey] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +------------------------------PhysicalProject +--------------------------------filter((part.p_brand = 'MFGR#2239')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_brand = 'MFGR#2239')) -------------------------------PhysicalOlapScan[part] +----------------------------filter((supplier.s_region = 'EUROPE')) +------------------------------PhysicalOlapScan[supplier] ----------------------PhysicalProject -------------------------filter((supplier.s_region = 'EUROPE')) ---------------------------PhysicalOlapScan[supplier] -------------------PhysicalProject ---------------------PhysicalOlapScan[dates] +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.2.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.2.out index 7d0b454caac1909..4ff658324ea03a5 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.2.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.2.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter((supplier.s_nation = 'UNITED STATES')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter((customer.c_nation = 'UNITED STATES')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter((supplier.s_nation = 'UNITED STATES')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter((customer.c_nation = 'UNITED STATES')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.3.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.3.out index 628f3df9a5831ad..e66f2bb1a8e6c9a 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.3.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.3.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_year <= 1997) and (dates.d_year >= 1992)) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.4.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.4.out index f725ccdbc2c1f7d..77a89293cd0d1c7 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.4.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q3.4.out @@ -4,24 +4,25 @@ PhysicalResultSink --PhysicalQuickSort[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 ---------------------------PhysicalProject -----------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) -------------------------------PhysicalOlapScan[supplier] -----------------------PhysicalProject -------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter((dates.d_yearmonth = 'Dec1997')) -----------------------PhysicalOlapScan[dates] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF1 c_custkey->[lo_custkey] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------PhysicalProject +------------------------------filter(s_city IN ('UNITED KI1', 'UNITED KI5')) +--------------------------------PhysicalOlapScan[supplier] +------------------------PhysicalProject +--------------------------filter(c_city IN ('UNITED KI1', 'UNITED KI5')) +----------------------------PhysicalOlapScan[customer] +--------------------PhysicalProject +----------------------filter((dates.d_yearmonth = 'Dec1997')) +------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.2.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.2.out index efc1e0061ed88d3..669575fcff515b1 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.2.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.2.out @@ -1,32 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF2 c_custkey->[lo_custkey] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF3 p_partkey->[lo_partkey] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF1 d_datekey->[lo_orderdate] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF2 c_custkey->[lo_custkey] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF1 d_datekey->[lo_orderdate] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((supplier.s_region = 'AMERICA')) +--------------------------------------PhysicalOlapScan[supplier] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_region = 'AMERICA')) -----------------------------------PhysicalOlapScan[supplier] +--------------------------------filter(d_year IN (1997, 1998)) +----------------------------------PhysicalOlapScan[dates] --------------------------PhysicalProject -----------------------------filter(d_year IN (1997, 1998)) -------------------------------PhysicalOlapScan[dates] +----------------------------filter((customer.c_region = 'AMERICA')) +------------------------------PhysicalOlapScan[customer] ----------------------PhysicalProject -------------------------filter((customer.c_region = 'AMERICA')) ---------------------------PhysicalOlapScan[customer] -------------------PhysicalProject ---------------------filter(p_mfgr IN ('MFGR#1', 'MFGR#2')) -----------------------PhysicalOlapScan[part] +------------------------filter(p_mfgr IN ('MFGR#1', 'MFGR#2')) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.3.out b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.3.out index 5bd9ad4782cb3d8..4a5c29f4ca3ca77 100644 --- a/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.3.out +++ b/regression-test/data/new_shapes_p0/ssb_sf100/shape/q4.3.out @@ -1,31 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF3 lo_custkey->[c_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] +--------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((lineorder.lo_custkey = customer.c_custkey)) otherCondition=() build RFs:RF3 lo_custkey->[c_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] apply RFs: RF3 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_orderdate = dates.d_datekey)) otherCondition=() build RFs:RF2 d_datekey->[lo_orderdate] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_partkey = part.p_partkey)) otherCondition=() build RFs:RF1 p_partkey->[lo_partkey] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((lineorder.lo_suppkey = supplier.s_suppkey)) otherCondition=() build RFs:RF0 s_suppkey->[lo_suppkey] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[lineorder] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((supplier.s_nation = 'UNITED STATES')) +--------------------------------------PhysicalOlapScan[supplier] ------------------------------PhysicalProject ---------------------------------filter((supplier.s_nation = 'UNITED STATES')) -----------------------------------PhysicalOlapScan[supplier] +--------------------------------filter((part.p_category = 'MFGR#14')) +----------------------------------PhysicalOlapScan[part] --------------------------PhysicalProject -----------------------------filter((part.p_category = 'MFGR#14')) -------------------------------PhysicalOlapScan[part] -----------------------PhysicalProject -------------------------filter(d_year IN (1997, 1998)) ---------------------------PhysicalOlapScan[dates] +----------------------------filter(d_year IN (1997, 1998)) +------------------------------PhysicalOlapScan[dates] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out index bc11b07f3a3809f..f8101b31831b813 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() -------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() +--------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalProject -----------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) -------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) +--------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out index 8cab83d94f65acf..491cb062ff6d133 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query11.out @@ -2,52 +2,55 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (2001, 2002)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (2001, 2002)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out index 10192bf86cb782e..b44b7db331182fb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +----------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] ---------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF15 +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +----------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] ---------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF18 +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out index 7b0fd9b10fd861c..3fbbc1ba6cd8c0c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] +----------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query17.out index c10cc616923d3c0..8608c5e3534c15a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF6 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF7 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter((d1.d_quarter_name = '2001Q1')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 RF6 +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF7 +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((d1.d_quarter_name = '2001Q1')) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query18.out index dcda7d5d7eb3ee2..2164dcd1cd27f3c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +--------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------PhysicalProject ---------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) -----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 +----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -----------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 1998)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query27.out index 886eca755706353..a8be055a16bb895 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query30.out index 6f1b848ed000345..7afe19d0cbde7aa 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'IN')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out index dad99a6b6465dd1..d9bc4269d81507c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out index a3f752e2ce0e3d5..b546a41de1481f7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() -------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() +--------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out index a083e5a72ef86a7..9848fde27974006 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query4.out @@ -2,73 +2,80 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF5 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id,customer_id,customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] ----------------------PhysicalProject ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF7 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF9 ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF7 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF9 --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out index 1465471817388b4..6385a642d8f3df1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalOlapScan[warehouse] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out index c6b4daafdce0df8..845b9ab5e2e03cb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] +----------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out index dc56d7f9e81411a..3903503ed08f163 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query50.out index 8bdf05df39ed987..5ed593b117c2edc 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +--------------------------------PhysicalOlapScan[store] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query6.out index 55144c3f93417ef..c9ab430dee54d42 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[item] ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query64.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query64.out index ac5d0d6d739e29a..2cff316c5a22aa0 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query64.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query64.out @@ -3,88 +3,90 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --PhysicalCteProducer ( cteId=CTEId#1 ) ----PhysicalProject -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[cr_item_sk,cs_item_sk,sr_item_sk,ss_item_sk] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[cr_item_sk,cs_item_sk,sr_item_sk,ss_item_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) ------------------------------------------------PhysicalProject ---------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() ----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() +------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() --------------------------------------------------------PhysicalProject -----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() +------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[sr_item_sk,ss_item_sk] +----------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() ------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +--------------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[sr_item_sk,ss_item_sk] ----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF6 RF19 +------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() +--------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF4 RF6 RF19 +--------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF4 RF19 ----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF4 RF19 -------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------filter((sale > (2 * refund))) -----------------------------------------------------------------------------hashAgg[GLOBAL] -------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------------------------------------------hashAgg[LOCAL] -----------------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() ---------------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF19 +------------------------------------------------------------------------------filter((sale > (2 * refund))) +--------------------------------------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------------------------------hashAgg[LOCAL] --------------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF19 +----------------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +------------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF19 +------------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF19 +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[customer] --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------PhysicalOlapScan[customer] +----------------------------------------------------------------------filter(d_year IN (2001, 2002)) +------------------------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------filter(d_year IN (2001, 2002)) ---------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------------PhysicalOlapScan[store] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------PhysicalOlapScan[store] +--------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------------------PhysicalProject ----------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------PhysicalProject -------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------------PhysicalProject --------------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------------------------PhysicalOlapScan[promotion] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[promotion] +------------------------------------------PhysicalOlapScan[household_demographics] ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[household_demographics] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[household_demographics] +----------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalOlapScan[income_band] --------------------PhysicalProject ----------------------PhysicalOlapScan[income_band] ----------------PhysicalProject -------------------PhysicalOlapScan[income_band] -------------PhysicalProject ---------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) -----------------PhysicalOlapScan[item] +------------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) +--------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query66.out index 596cb44922b54a8..448a30647ab9671 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query69.out index e0bbeea823735d7..b1a6d0836d81fd7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() -----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() +------------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF5 ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalProject ---------------------------filter(ca_state IN ('MI', 'TX', 'VA')) -----------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalProject +----------------------------filter(ca_state IN ('MI', 'TX', 'VA')) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query70.out index 9c9d7b7638d6a5a..41c315a3884d8a6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out index 2f21640b0799298..72561c50f3d29ad 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query81.out index 8e795ffc2e0eda2..8db7858d58f48fb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/noStatsRfPrune/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'CA')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out index 36a2c69f6b95167..352ee09144aa9a7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] -------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalProject -----------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) -------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) +--------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out index e963fc6a8caa7a3..035bb9d52145278 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query11.out @@ -2,52 +2,55 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ss_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (2001, 2002)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (2001, 2002)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out index 966f8701126465b..1c57f1e9910105a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] ---------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF12 d_date_sk->[ss_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF11 i_item_sk->[ss_item_sk,ss_item_sk] +------------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF10 ss_item_sk->[ss_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF11 +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk,i_item_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF15 d_date_sk->[cs_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF13 i_item_sk->[cs_item_sk] +----------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk,i_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] apply RFs: RF14 ---------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF13 i_item_sk->[cs_item_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[item] apply RFs: RF14 +------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[i_item_sk,ws_item_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF18 d_date_sk->[ws_sold_date_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF16 i_item_sk->[ws_item_sk] +----------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[i_item_sk,ws_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[item] apply RFs: RF17 ---------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF16 i_item_sk->[ws_item_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[item] apply RFs: RF17 +------------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out index 1fba646d62b40ab..df3c45a2d384c3d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF1 ca_address_sk->[c_current_addr_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[cs_bill_customer_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF2 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] apply RFs: RF1 --------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------PhysicalOlapScan[customer_address] ----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -------------------PhysicalProject ---------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query17.out index 52da90d84ff3a84..3c56047c83427e9 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF6 RF9 ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF5 RF7 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalProject -----------------------------------filter((d1.d_quarter_name = '2001Q1')) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF9 s_store_sk->[ss_store_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[sr_returned_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF3 cs_bill_customer_sk->[sr_customer_sk,ss_customer_sk];RF4 cs_item_sk->[sr_item_sk,ss_item_sk] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_customer_sk->[ss_customer_sk];RF1 sr_item_sk->[ss_item_sk];RF2 sr_ticket_number->[ss_ticket_number] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 RF5 RF6 RF9 +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF3 RF4 RF5 RF7 +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF8 +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[item] +----------------------------------PhysicalProject +------------------------------------filter((d1.d_quarter_name = '2001Q1')) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query18.out index 22f67b07a4698c3..68d3302b171fe07 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF4 i_item_sk->[cs_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[c_current_cdemo_sk] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[cs_bill_customer_sk] ------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF5 +--------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[cs_bill_cdemo_sk] +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 RF5 +----------------------------------------------PhysicalProject +------------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +--------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------PhysicalProject ---------------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) -----------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -----------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter((date_dim.d_year = 1998)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 1998)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query27.out index 3eec2f7437212bd..0aac3cae957c5fd 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ss_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ss_item_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query30.out index 6671347af5cc6cc..89847451ae9a672 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'IN')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out index 25429183c8fe4a9..44f18ab47b8e229 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) ---------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] -------------PhysicalProject ---------------PhysicalOlapScan[customer] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +----------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out index 14b71f576d97e0b..cf862f42ab5497f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] -------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] +--------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out index 9a590246f64a4a1..13b5871c5779cf2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query4.out @@ -2,73 +2,80 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ss_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ws_sold_date_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ws_bill_customer_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] +------------------PhysicalProject +--------------------filter(d_year IN (1999, 2000)) +----------------------PhysicalOlapScan[date_dim] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id] --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 --------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id,customer_id,customer_id] +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] ----------------------PhysicalProject ------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF7 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF9 ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF7 +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF9 --------------------------PhysicalProject ----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out index aae0d7885570451..5828a951371dbef 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[cs_warehouse_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +--------------------------------hashJoin[LEFT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_returns] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalOlapScan[warehouse] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] +----------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +------------------------------PhysicalOlapScan[item] ----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out index 038ece83d7c0447..023efb28243d89e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF0 c_customer_sk->[ws_bill_customer_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalProject +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out index dc56d7f9e81411a..3903503ed08f163 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF0 wr_order_number->[ws_order_number];RF1 wr_item_sk->[ws_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF3 cr_order_number->[cs_order_number];RF4 cr_item_sk->[cs_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 RF5 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[ss_sold_date_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF6 sr_ticket_number->[ss_ticket_number];RF7 sr_item_sk->[ss_item_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 RF8 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) -------------------------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query50.out index e7941ce875f02f7..e1fb5c9828d596d 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[sr_returned_date_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF0 sr_ticket_number->[ss_ticket_number];RF1 sr_item_sk->[ss_item_sk];RF2 sr_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 RF4 +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF5 +--------------------------------PhysicalOlapScan[store] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store] +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------PhysicalOlapScan[date_dim] +------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query54.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query54.out index 96d57c63e6f62bd..f1544cf0b3cd60e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query54.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query54.out @@ -17,7 +17,7 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[ss_sold_date_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF5 s_county->[ca_county];RF6 s_state->[ca_state] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF5 s_state->[ca_state];RF6 s_county->[ca_county] ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF4 ca_address_sk->[c_current_addr_sk] ----------------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query6.out index 5f53b66408a5004..ca5b3a32a707574 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF5 i_category->[i_category] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF4 d_month_seq->[d_month_seq] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] apply RFs: RF5 +----------------------------PhysicalAssertNumRows +------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------hashAgg[LOCAL] --------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF4 +----------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------hashAgg[LOCAL] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] apply RFs: RF5 ---------------------------PhysicalAssertNumRows -----------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query64.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query64.out index b7cf8115b6db75a..d1d49f78ba895b8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query64.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query64.out @@ -3,88 +3,90 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --PhysicalCteProducer ( cteId=CTEId#1 ) ----PhysicalProject -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[cr_item_sk,cs_item_sk,sr_item_sk,ss_item_sk] +------hashAgg[GLOBAL] +--------PhysicalDistribute[DistributionSpecHash] +----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF18 ib_income_band_sk->[hd_income_band_sk] +--------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF19 i_item_sk->[cr_item_sk,cs_item_sk,sr_item_sk,ss_item_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF17 ib_income_band_sk->[hd_income_band_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd2.hd_income_band_sk = ib2.ib_income_band_sk)) otherCondition=() build RFs:RF18 ib_income_band_sk->[hd_income_band_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF16 ca_address_sk->[c_current_addr_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((hd1.hd_income_band_sk = ib1.ib_income_band_sk)) otherCondition=() build RFs:RF17 ib_income_band_sk->[hd_income_band_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF15 ca_address_sk->[ss_addr_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = ad2.ca_address_sk)) otherCondition=() build RFs:RF16 ca_address_sk->[c_current_addr_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF14 hd_demo_sk->[c_current_hdemo_sk] +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = ad1.ca_address_sk)) otherCondition=() build RFs:RF15 ca_address_sk->[ss_addr_sk] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF13 hd_demo_sk->[ss_hdemo_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_hdemo_sk = hd2.hd_demo_sk)) otherCondition=() build RFs:RF14 hd_demo_sk->[c_current_hdemo_sk] ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF12 p_promo_sk->[ss_promo_sk] +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = hd1.hd_demo_sk)) otherCondition=() build RFs:RF13 hd_demo_sk->[ss_hdemo_sk] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF11 cd_demo_sk->[c_current_cdemo_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_promo_sk = promotion.p_promo_sk)) otherCondition=() build RFs:RF12 p_promo_sk->[ss_promo_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF10 cd_demo_sk->[ss_cdemo_sk] +----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=(( not (cd_marital_status = cd_marital_status))) build RFs:RF11 cd_demo_sk->[c_current_cdemo_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[c_first_shipto_date_sk] +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF10 cd_demo_sk->[ss_cdemo_sk] ----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[c_first_sales_date_sk] +------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_shipto_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF9 d_date_sk->[c_first_shipto_date_sk] --------------------------------------------------------PhysicalProject -----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF7 s_store_sk->[ss_store_sk] +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_first_sales_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF8 d_date_sk->[c_first_sales_date_sk] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF7 s_store_sk->[ss_store_sk] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] +------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[sr_item_sk,ss_item_sk] +----------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ss_customer_sk] ------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_item_sk->[ss_item_sk];RF3 sr_ticket_number->[ss_ticket_number] +--------------------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cs_ui.cs_item_sk)) otherCondition=() build RFs:RF4 cs_item_sk->[sr_item_sk,ss_item_sk] ----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 RF10 RF12 RF13 RF15 RF19 +------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_item_sk->[ss_item_sk];RF3 sr_ticket_number->[ss_ticket_number] +--------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 RF4 RF5 RF6 RF7 RF10 RF12 RF13 RF15 RF19 +--------------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF4 RF19 ----------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF4 RF19 -------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------filter((sale > (2 * refund))) -----------------------------------------------------------------------------hashAgg[GLOBAL] -------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------------------------------------------------hashAgg[LOCAL] -----------------------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_item_sk->[cs_item_sk];RF1 cr_order_number->[cs_order_number] ---------------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF19 +------------------------------------------------------------------------------filter((sale > (2 * refund))) +--------------------------------------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------------------------------------hashAgg[LOCAL] --------------------------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF19 +----------------------------------------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF0 cr_item_sk->[cs_item_sk];RF1 cr_order_number->[cs_order_number] +------------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF19 +------------------------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF19 +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF8 RF9 RF11 RF14 RF16 --------------------------------------------------------------------PhysicalProject -----------------------------------------------------------------------PhysicalOlapScan[customer] apply RFs: RF8 RF9 RF11 RF14 RF16 +----------------------------------------------------------------------filter(d_year IN (2001, 2002)) +------------------------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------filter(d_year IN (2001, 2002)) ---------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------------PhysicalOlapScan[store] ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------PhysicalOlapScan[store] +--------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------------------------PhysicalProject ----------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------PhysicalProject -------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------------PhysicalProject --------------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------------------------PhysicalOlapScan[promotion] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[promotion] +------------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF17 ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF17 +--------------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF18 --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[household_demographics] apply RFs: RF18 +----------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalOlapScan[income_band] --------------------PhysicalProject ----------------------PhysicalOlapScan[income_band] ----------------PhysicalProject -------------------PhysicalOlapScan[income_band] -------------PhysicalProject ---------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) -----------------PhysicalOlapScan[item] +------------------filter((item.i_current_price <= 33.00) and (item.i_current_price >= 24.00) and i_color IN ('blanched', 'brown', 'burlywood', 'chocolate', 'drab', 'medium')) +--------------------PhysicalOlapScan[item] --PhysicalResultSink ----PhysicalQuickSort[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query66.out index 7ef36371976ad65..30e394ec0060293 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[warehouse] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------PhysicalOlapScan[ship_mode] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF3 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF0 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF7 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[warehouse] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query69.out index af6d7e8c85a5f6c..489c7c60182b8b7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] -----------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF5 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] +------------------------------hashJoin[LEFT_ANTI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF4 RF5 ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalProject ---------------------------filter(ca_state IN ('MI', 'TX', 'VA')) -----------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalProject +----------------------------filter(ca_state IN ('MI', 'TX', 'VA')) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query70.out index 866c026a90dd671..4614292ea19fd11 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +----------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[RIGHT_SEMI_JOIN bucketShuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out index 8f739a1d12b35a4..1140e5137f45fd5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 i_item_sk->[ws_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query81.out index 8006799a3cc5fd1..0286c6e8566c944 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/no_stats_shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year = 2002)) --------------------PhysicalOlapScan[date_dim] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF3 -------------------PhysicalProject ---------------------filter((customer_address.ca_state = 'CA')) -----------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ctr_customer_sk] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF2 RF4 +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out index 6027b8b2684b4b7..4de40406337c3b3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ---------------------------------------PhysicalOlapScan[customer_address] +--------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query11.out index e7ae73f8e009807..122456d14826a27 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query11.out @@ -2,53 +2,56 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter(d_year IN (2001, 2002)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 +----------------------PhysicalProject +------------------------filter(d_year IN (2001, 2002)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] ---------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] ---------------------PhysicalProject -----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id,customer_id,customer_id] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 +--------------------PhysicalProject +----------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 +----------------PhysicalProject +------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out index 2a29746e37ef076..60890e71af57121 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +--------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +--------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------------------PhysicalOlapScan[item] +--------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out index f15477f65e12fb1..fdef33724c6bab4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query17.out index 5342955a97aae22..ad361c2f1e04a7c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 -------------------------------------PhysicalProject ---------------------------------------filter((d1.d_quarter_name = '2001Q1')) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 -------------------------------------PhysicalProject ---------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_quarter_name = '2001Q1')) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query18.out index 57183675eb5fc2f..0929ca1a0ece88c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query27.out index 9b311ff91423bf1..322e3efe5d76d3c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query30.out index 99f54520a25e121..9f2571fe3f97a07 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out index bb286ee190e816a..2295c980cdd3200 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalProject ---------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) -----------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +----------------------------------PhysicalOlapScan[household_demographics] +--------------------------PhysicalProject +----------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out index 4da981f140aa0e0..a569909a86c5fbc 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_demographics] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF5 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out index 980ceef87cedc27..9e2ef78ab7ea161 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query4.out @@ -2,74 +2,81 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] ---PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() ------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] -----------------------------PhysicalProject -------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id,customer_id,customer_id,customer_id,customer_id] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF8 customer_id->[customer_id,customer_id,customer_id] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF8 RF9 RF10 +----------------------------PhysicalProject +------------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF8 RF9 RF10 +------------------------PhysicalProject +--------------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 RF10 +--------------------PhysicalProject +----------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 RF10 +------------------PhysicalProject +--------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out index ade38048fb9732f..a2b334f865cad43 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 +----------------------------------PhysicalProject +------------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +--------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out index 83f6b9ca5df1f0a..02dea1281d1065c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out index e34962e0847b0dd..5a3e4ad4e0d79ed 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query50.out index 2f0a1b10cbff1b7..5108634440efba4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 ------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query54.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query54.out index ca44d791dc42aac..3f16ecb0726d33a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query54.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query54.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_state->[ca_state];RF5 s_county->[ca_county] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query6.out index ddb58c3887ed6fe..79795fd8b38e1c2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query66.out index 6017253a3383c5d..15f9b57a795b2d8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query69.out index a68ff0c1138094c..c78152cc022c247 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] -------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) -----------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) +------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query70.out index ae5b26647980e76..546713bec968de2 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out index 668c3625c568417..5b2d657de9c18f1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF2 -----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] apply RFs: RF3 +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query81.out index fdc3edc4efb31fd..0dafcf2058004aa 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'CA')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query91.out b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query91.out index e2a4ed89af442da..8eb26ba170c02fe 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query91.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/rf_prune/query91.out @@ -9,13 +9,13 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cr_returned_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cr_returned_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returning_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cr_returning_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 RF4 +------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF3 RF5 ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 c_current_addr_sk->[ca_address_sk] --------------------------------PhysicalProject @@ -34,8 +34,8 @@ PhysicalResultSink --------------------------------------filter((hd_buy_potential like '1001-5000%')) ----------------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject ---------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[call_center] --------------------PhysicalProject -----------------------PhysicalOlapScan[call_center] +----------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out index 6027b8b2684b4b7..4de40406337c3b3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) ---------------------------------------PhysicalOlapScan[customer_address] +--------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 4) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Cochran County', 'Kandiyohi County', 'Marquette County', 'Storey County', 'Warren County')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query11.out index 8abb7de87e97f9c..ff129e700362f76 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query11.out @@ -2,53 +2,56 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (2001, 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------filter(d_year IN (2001, 2002)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ws_bill_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------filter(d_year IN (2001, 2002)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] ---------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] ---------------------PhysicalProject -----------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id,customer_id,customer_id] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 2001) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 +--------------------PhysicalProject +----------------------filter((t_w_firstyear.dyear = 2001) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 +----------------PhysicalProject +------------------filter((t_s_secyear.dyear = 2002) and (t_s_secyear.sale_type = 's')) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2002) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out index 196a98b5a2f51d3..386242af1221bdd 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query14.out @@ -71,84 +71,86 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2002) and (date_dim.d_year >= 2000)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF15 ss_item_sk->[cs_item_sk,i_item_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk] +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF15 ss_item_sk->[cs_item_sk,i_item_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF14 i_item_sk->[cs_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +--------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] apply RFs: RF15 -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------------------------PhysicalOlapScan[item] apply RFs: RF15 +--------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF18 ss_item_sk->[i_item_sk,ws_item_sk] +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ws_item_sk] +------------------------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF18 ss_item_sk->[i_item_sk,ws_item_sk] --------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +----------------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF17 i_item_sk->[ws_item_sk] ------------------------------------------------PhysicalProject ---------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) +--------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[item] apply RFs: RF18 -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------------------------------------PhysicalOlapScan[item] apply RFs: RF18 +--------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out index b80017b7ebd0f9a..f7588aafe93615b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query17.out index 7cc4a196c206c39..11cff6336d28e48 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 -------------------------------------PhysicalProject ---------------------------------------filter((d1.d_quarter_name = '2001Q1')) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 -------------------------------------PhysicalProject ---------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF6 s_store_sk->[ss_store_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[sr_item_sk,ss_item_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_quarter_name = '2001Q1')) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF5 +--------------------------------------PhysicalProject +----------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query18.out index 57183675eb5fc2f..0929ca1a0ece88c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------PhysicalOlapScan[item] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 cs_item_sk->[i_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalOlapScan[item] apply RFs: RF5 +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Advanced Degree') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1998)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 2, 4, 7, 8)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('GA', 'IN', 'ME', 'NC', 'OK', 'WA', 'WY')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query27.out index c6137e774ae1b93..126222e256bdbb8 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[ss_item_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'F') and (customer_demographics.cd_marital_status = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalProject -------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) ---------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('AL', 'LA', 'MI', 'MO', 'SC', 'TN')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query30.out index 1fd2b5a1688c125..5db7e0e9a2854b7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'IN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'IN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out index bb286ee190e816a..2295c980cdd3200 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) -------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalProject ---------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) -----------------------------PhysicalOlapScan[store] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN shuffle] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF2 s_store_sk->[ss_store_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF1 hd_demo_sk->[ss_hdemo_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (1998, 1999, 2000)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +----------------------------------PhysicalOlapScan[household_demographics] +--------------------------PhysicalProject +----------------------------filter(s_county IN ('Barrow County', 'Daviess County', 'Franklin Parish', 'Luce County', 'Richland County', 'Walker County', 'Williamson County', 'Ziebach County')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out index 93fdd630b3352d8..004e074415dd697 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 RF4 RF5 -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_demographics] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--------------------------hashJoin[LEFT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF5 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF4 cd_demo_sk->[c_current_cdemo_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF3 RF4 RF5 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_address] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out index 709da33d851bffc..331905fe3e15248 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query4.out @@ -2,74 +2,81 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] ---PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] -----------------------------PhysicalProject -------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id,customer_id,customer_id,customer_id,customer_id] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF8 customer_id->[customer_id,customer_id,customer_id] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF8 RF9 RF10 +----------------------------PhysicalProject +------------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF8 RF9 RF10 +------------------------PhysicalProject +--------------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 RF10 +--------------------PhysicalProject +----------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 RF10 +------------------PhysicalProject +--------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out index 5ff27658e2ed3ff..9b20475ecb8d866 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +----------------------------------PhysicalProject +------------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +--------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------------filter((date_dim.d_date <= '2001-05-02') and (date_dim.d_date >= '2001-03-03')) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out index e05c3f0537a669e..c524f20d35f451f 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out index e34962e0847b0dd..5a3e4ad4e0d79ed 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 12) and (date_dim.d_year = 1999)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query50.out index f5c3f38463d42ca..b45ce616bee1d2a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query54.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query54.out index c2d65d63990faef..1fb67a2b9eae2b5 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query54.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query54.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_state->[ca_state];RF5 s_county->[ca_county] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query6.out index 43c8732d7f8553f..18dbe974af8c0c4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query66.out index a4ff984e4cdc434..fedc1cffa270c1e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 1998)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 77621) and (time_dim.t_time >= 48821)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query69.out index a68ff0c1138094c..c78152cc022c247 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) ------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] -------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +--------------------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) -----------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter(ca_state IN ('MI', 'TX', 'VA')) +------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query70.out index b1074dc6ffeed5b..6f19921b730a2aa 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1224) and (d1.d_month_seq >= 1213)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------filter((date_dim.d_month_seq <= 1224) and (date_dim.d_month_seq >= 1213)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out index 668c3625c568417..5b2d657de9c18f1 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] apply RFs: RF3 -------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------PhysicalProject -------------------------------filter(ss_hdemo_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF2 -----------------------------PhysicalProject -------------------------------filter(cs_warehouse_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 ss_sold_date_sk->[d_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] apply RFs: RF3 +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 ss_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------PhysicalProject +--------------------------------filter(ss_hdemo_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute[DistributionSpecExecutionAny] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 cs_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF2 +------------------------------PhysicalProject +--------------------------------filter(cs_warehouse_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query81.out index e6aef6266d392b3..1ccd679f36f2930 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'CA')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'CA')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query91.out b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query91.out index 6f3970cbb29d50e..bc2eac3de8aa780 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query91.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf100/shape/query91.out @@ -9,9 +9,9 @@ PhysicalResultSink ------------PhysicalDistribute[DistributionSpecHash] --------------hashAgg[LOCAL] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF5 cc_call_center_sk->[cr_call_center_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cr_returned_date_sk] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returned_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cr_returned_date_sk] +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk)) otherCondition=() build RFs:RF4 cc_call_center_sk->[cr_call_center_sk] ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_returns.cr_returning_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cr_returning_customer_sk] ----------------------------PhysicalProject @@ -34,8 +34,8 @@ PhysicalResultSink --------------------------------------filter((hd_buy_potential like '1001-5000%')) ----------------------------------------PhysicalOlapScan[household_demographics] ------------------------PhysicalProject ---------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[call_center] --------------------PhysicalProject -----------------------PhysicalOlapScan[call_center] +----------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out index 6ac3b85090acebc..b24d9db64b18123 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query54.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query54.out index 397a41b34c4e60b..2c1e5cdc05399ca 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query54.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query54.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_state->[ca_state];RF5 s_county->[ca_county] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query6.out index 43c8732d7f8553f..18dbe974af8c0c4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/bs_downgrade_shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out index f68d46db52fbb83..a6e5ab7c60edd6c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/eliminate_empty/query10_empty.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) ---------------------------------------PhysicalOlapScan[customer_address] +--------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out index f68d46db52fbb83..a6e5ab7c60edd6c 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query10.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF5 ----------------------------PhysicalProject ------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) --------------------------------PhysicalOlapScan[date_dim] -------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +------------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 ------------------------------PhysicalProject --------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------------------PhysicalProject -------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) ---------------------------------------PhysicalOlapScan[customer_address] +--------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ss_customer_sk] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 6) and (date_dim.d_moy >= 3) and (date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] +--------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------PhysicalProject +--------------------------------------filter(ca_county IN ('Campbell County', 'Cleburne County', 'Escambia County', 'Fairfield County', 'Washtenaw County')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query11.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query11.out index 35504b7f44d24e2..a6c7f181e5ad8ff 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query11.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query11.out @@ -2,53 +2,56 @@ -- !ds_shape_11 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1998, 1999)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 -------------------------PhysicalProject ---------------------------filter(d_year IN (1998, 1999)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------filter(d_year IN (1998, 1999)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ws_bill_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------filter(d_year IN (1998, 1999)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] --PhysicalResultSink ----PhysicalTopN[MERGE_SORT] ------PhysicalDistribute[DistributionSpecGather] --------PhysicalTopN[LOCAL_SORT] ----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF5 customer_id->[customer_id] ---------------PhysicalProject -----------------filter((t_w_secyear.dyear = 1999) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------PhysicalProject -----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id,customer_id] -------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF3 customer_id->[customer_id] ---------------------PhysicalProject -----------------------filter((t_s_secyear.dyear = 1999) and (t_s_secyear.sale_type = 's')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000) > if((year_total > 0.00), (cast(year_total as DECIMALV3(38, 8)) / year_total), 0.000000))) build RFs:RF6 customer_id->[customer_id,customer_id,customer_id] +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] --------------------PhysicalProject ----------------------filter((t_s_firstyear.dyear = 1998) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.00)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1998) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 RF6 +--------------------PhysicalProject +----------------------filter((t_w_firstyear.dyear = 1998) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.00)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 RF6 +----------------PhysicalProject +------------------filter((t_s_secyear.dyear = 1999) and (t_s_secyear.sale_type = 's')) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 1999) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out index 61f29b112113463..6e6c71f54a16d54 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query14.out @@ -71,82 +71,84 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------filter((date_dim.d_year <= 2001) and (date_dim.d_year >= 1999)) --------------------PhysicalOlapScan[date_dim] ----PhysicalResultSink -------PhysicalTopN[MERGE_SORT] ---------PhysicalDistribute[DistributionSpecGather] -----------PhysicalTopN[LOCAL_SORT] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalRepeat -----------------------PhysicalUnion -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +------PhysicalProject +--------PhysicalTopN[MERGE_SORT] +----------PhysicalDistribute[DistributionSpecGather] +------------PhysicalTopN[LOCAL_SORT] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------PhysicalRepeat +--------------------------PhysicalUnion ----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] -----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF12 i_item_sk->[ss_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF11 ss_item_sk->[ss_item_sk] ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF10 d_date_sk->[ss_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF10 RF11 RF12 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF12 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------------PhysicalProject ---------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] -----------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF15 i_item_sk->[cs_item_sk,ss_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF14 ss_item_sk->[cs_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF13 d_date_sk->[cs_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF13 RF14 RF15 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF15 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------------PhysicalProject -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute[DistributionSpecGather] -----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +--------------------------------PhysicalProject +----------------------------------hashAgg[GLOBAL] +------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------hashAgg[LOCAL] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF18 i_item_sk->[ss_item_sk,ws_item_sk] +--------------------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = cross_items.ss_item_sk)) otherCondition=() build RFs:RF17 ss_item_sk->[ws_item_sk] +----------------------------------------------PhysicalProject +------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF16 d_date_sk->[ws_sold_date_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF16 RF17 RF18 +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 2001)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF18 +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalProject +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out index 6f522756f180982..4bb1815ede44778 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query15.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------hashJoin[INNER_JOIN shuffle] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=(((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR ca_state IN ('CA', 'GA', 'WA')) OR (catalog_sales.cs_sales_price > 500.00))) build RFs:RF2 c_customer_sk->[cs_bill_customer_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF2 +--------------------------PhysicalProject +----------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 2) and (date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF0 -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query17.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query17.out index 12fa11701b619f7..34fa633896f04d4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query17.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query17.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 -------------------------PhysicalProject ---------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 -------------------------------------PhysicalProject ---------------------------------------filter((d1.d_quarter_name = '2001Q1')) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF6 -------------------------------------PhysicalProject ---------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) and (store_returns.sr_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF8 sr_customer_sk->[cs_bill_customer_sk];RF9 sr_item_sk->[cs_item_sk] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = d3.d_date_sk)) otherCondition=() build RFs:RF7 d_date_sk->[cs_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF7 RF8 RF9 +--------------------------PhysicalProject +----------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = store_sales.ss_item_sk)) otherCondition=() build RFs:RF6 i_item_sk->[sr_item_sk,ss_item_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF2 sr_customer_sk->[ss_customer_sk];RF3 sr_item_sk->[ss_item_sk];RF4 sr_ticket_number->[ss_ticket_number] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 RF6 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_quarter_name = '2001Q1')) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 RF6 +--------------------------------------PhysicalProject +----------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query18.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query18.out index ea401d9c36dc081..1b60930683fe9b6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query18.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query18.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF5 i_item_sk->[cs_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 -----------------------------------PhysicalProject -------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk)) otherCondition=() build RFs:RF2 cd_demo_sk->[cs_bill_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 RF4 RF5 +--------------------------------------PhysicalProject +----------------------------------------filter((cd1.cd_education_status = 'Primary') and (cd1.cd_gender = 'F')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((customer.c_current_cdemo_sk = cd2.cd_demo_sk)) otherCondition=() build RFs:RF1 c_current_cdemo_sk->[cd_demo_sk] --------------------------------------PhysicalProject -----------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) -------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +----------------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF1 --------------------------------------PhysicalProject -----------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) -------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 10, 11, 3, 4, 7)) +----------------------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------------------PhysicalProject +--------------------------------------------filter(ca_state IN ('AL', 'CA', 'GA', 'IN', 'MO', 'MT', 'TN')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2001)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query27.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query27.out index 0fa387fb0d6bb92..7230028e9e726ef 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query27.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query27.out @@ -1,33 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalRepeat +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------PhysicalRepeat ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk)) otherCondition=() build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) +------------------------------------------PhysicalOlapScan[customer_demographics] ----------------------------------PhysicalProject -------------------------------------filter((customer_demographics.cd_education_status = 'Secondary') and (customer_demographics.cd_gender = 'M') and (customer_demographics.cd_marital_status = 'W')) ---------------------------------------PhysicalOlapScan[customer_demographics] +------------------------------------filter((store.s_state = 'TN')) +--------------------------------------PhysicalOlapScan[store] ------------------------------PhysicalProject ---------------------------------filter((store.s_state = 'TN')) -----------------------------------PhysicalOlapScan[store] +--------------------------------filter((date_dim.d_year = 1999)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] +----------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query30.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query30.out index c894fcceff19a5b..d88a3aa78d8a741 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query30.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query30.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'AR')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'AR')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out index 135b461a1447621..c11c22f5ba316ba 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query34.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------PhysicalProject -----------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] -------------PhysicalProject ---------------PhysicalOlapScan[customer] apply RFs: RF3 -------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 ---------------------------------PhysicalProject -----------------------------------filter((store.s_county = 'Williamson County')) -------------------------------------PhysicalOlapScan[store] -----------------------------PhysicalProject -------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) -----------------------------PhysicalOlapScan[household_demographics] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN broadcast] hashCondition=((dn.ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 ss_customer_sk->[c_customer_sk] +--------------PhysicalProject +----------------PhysicalOlapScan[customer] apply RFs: RF3 +--------------filter((dn.cnt <= 20) and (dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk)) otherCondition=() build RFs:RF2 hd_demo_sk->[ss_hdemo_sk] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF0 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 RF2 +----------------------------------PhysicalProject +------------------------------------filter((store.s_county = 'Williamson County')) +--------------------------------------PhysicalOlapScan[store] +------------------------------PhysicalProject +--------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28))) and d_year IN (2000, 2001, 2002)) +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------filter((household_demographics.hd_vehicle_count > 0) and (if((hd_vehicle_count > 0), (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)), NULL) > 1.2) and hd_buy_potential IN ('0-500', '1001-5000')) +------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out index a7a8d54bd3e1359..fc464a9984ddeb3 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query35.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) ---------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] ---------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() -----------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((ifnull($c$1, FALSE) OR ifnull($c$2, FALSE))) +----------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF5 cd_demo_sk->[c_current_cdemo_sk] +----------------------------hashJoin[LEFT_SEMI_JOIN bucketShuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[ss_customer_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF5 +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ----------------------------------PhysicalProject ------------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) --------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF5 -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 ---------------------------PhysicalProject -----------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 +----------------------------PhysicalProject +------------------------------filter((date_dim.d_qoy < 4) and (date_dim.d_year = 1999)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out index 709da33d851bffc..331905fe3e15248 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query4.out @@ -2,74 +2,81 @@ -- !ds_shape_4 -- PhysicalCteAnchor ( cteId=CTEId#0 ) --PhysicalCteProducer ( cteId=CTEId#0 ) -----PhysicalProject -------hashJoin[INNER_JOIN shuffle] hashCondition=((ss_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk,ss_customer_sk,ws_bill_customer_sk] ---------PhysicalProject -----------PhysicalUnion -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF1 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ws_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] apply RFs: RF2 RF3 -------------------------PhysicalProject ---------------------------filter(d_year IN (1999, 2000)) -----------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------PhysicalOlapScan[customer] ---PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF8 customer_id->[customer_id] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF1 c_customer_sk->[ss_customer_sk] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id] +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = catalog_sales.cs_bill_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[cs_bill_customer_sk] ------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[cs_sold_date_sk] +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +----------------------PhysicalProject +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute[DistributionSpecHash] +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() build RFs:RF5 c_customer_sk->[ws_bill_customer_sk] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF6 customer_id->[customer_id] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ws_sold_date_sk] ----------------------PhysicalProject -------------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 +------------------------PhysicalOlapScan[web_sales] apply RFs: RF4 RF5 ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF5 customer_id->[customer_id,customer_id] ---------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF4 customer_id->[customer_id] -----------------------------PhysicalProject -------------------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 RF5 +------------------------filter(d_year IN (1999, 2000)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] +--PhysicalResultSink +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF10 customer_id->[customer_id,customer_id,customer_id,customer_id,customer_id] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_c_secyear.customer_id)) otherCondition=((if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL) > if((year_total > 0.000000), (cast(year_total as DECIMALV3(38, 16)) / year_total), NULL))) build RFs:RF9 customer_id->[customer_id,customer_id,customer_id,customer_id] +------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_secyear.customer_id = t_s_firstyear.customer_id)) otherCondition=() build RFs:RF8 customer_id->[customer_id,customer_id,customer_id] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((t_s_firstyear.customer_id = t_w_firstyear.customer_id)) otherCondition=() build RFs:RF7 customer_id->[customer_id,customer_id] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((t_s_firstyear.customer_id = t_c_firstyear.customer_id)) otherCondition=() build RFs:RF6 customer_id->[customer_id] ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.dyear = 1999) and (t_s_firstyear.sale_type = 's') and (t_s_firstyear.year_total > 0.000000)) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF5 ---------------------------PhysicalProject -----------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF6 RF7 RF8 RF9 RF10 +----------------------------PhysicalProject +------------------------------filter((t_c_firstyear.dyear = 1999) and (t_c_firstyear.sale_type = 'c') and (t_c_firstyear.year_total > 0.000000)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF7 RF8 RF9 RF10 +------------------------PhysicalProject +--------------------------filter((t_w_firstyear.dyear = 1999) and (t_w_firstyear.sale_type = 'w') and (t_w_firstyear.year_total > 0.000000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF8 RF9 RF10 +--------------------PhysicalProject +----------------------filter((t_s_secyear.dyear = 2000) and (t_s_secyear.sale_type = 's')) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF9 RF10 +------------------PhysicalProject +--------------------filter((t_c_secyear.dyear = 2000) and (t_c_secyear.sale_type = 'c')) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF10 +--------------PhysicalProject +----------------filter((t_w_secyear.dyear = 2000) and (t_w_secyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out index 041e5711184598a..4f8e7f620044d0b 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query40.out @@ -1,30 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF4 w_warehouse_sk->[cs_warehouse_sk] ----------------------PhysicalProject -------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[RIGHT_OUTER_JOIN colocated] hashCondition=((catalog_sales.cs_item_sk = catalog_returns.cr_item_sk) and (catalog_sales.cs_order_number = catalog_returns.cr_order_number)) otherCondition=() build RFs:RF2 cs_order_number->[cr_order_number];RF3 cs_item_sk->[cr_item_sk] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF2 RF3 --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[cs_sold_date_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((item.i_item_sk = catalog_sales.cs_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[cs_item_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF0 RF1 RF4 +----------------------------------PhysicalProject +------------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) +--------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalProject ---------------------------------filter((item.i_current_price <= 1.49) and (item.i_current_price >= 0.99)) -----------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +--------------------------------filter((date_dim.d_date <= '2001-06-01') and (date_dim.d_date >= '2001-04-02')) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out index 6ac3b85090acebc..b24d9db64b18123 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query45.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------filter((substring(ca_zip, 1, 5) IN ('80348', '81792', '83405', '85392', '85460', '85669', '86197', '86475', '88274') OR $c$1)) +----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ws_item_sk] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------hashJoin[INNER_JOIN shuffle] hashCondition=((web_sales.ws_bill_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 c_customer_sk->[ws_bill_customer_sk] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF1 RF2 RF3 +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy = 1) and (date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((customer.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF0 ca_address_sk->[c_current_addr_sk] +--------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] apply RFs: RF0 +------------------------------PhysicalOlapScan[item] ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((item.i_item_id = item.i_item_id)) otherCondition=() -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] -------------------------PhysicalProject ---------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) -----------------------------PhysicalOlapScan[item] +------------------------------filter(i_item_sk IN (11, 13, 17, 19, 2, 23, 29, 3, 5, 7)) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query48.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query48.out index 3fcfef0d8f68ba8..13d52b16b1f0cf6 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query48.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query48.out @@ -7,9 +7,9 @@ PhysicalResultSink --------PhysicalProject ----------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] ------------PhysicalProject ---------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------hashJoin[INNER_JOIN shuffleBucket] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('ND', 'NY', 'SD') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('GA', 'KS', 'MD') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('CO', 'MN', 'NC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF2 ca_address_sk->[ss_addr_sk] ----------------PhysicalProject -------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_addr_sk = customer_address.ca_address_sk)) otherCondition=((((ca_state IN ('ND', 'NY', 'SD') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('GA', 'KS', 'MD') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('CO', 'MN', 'NC') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00))))) build RFs:RF1 ca_address_sk->[ss_addr_sk] +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ss_sold_date_sk] --------------------PhysicalProject ----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)) otherCondition=((((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))))) build RFs:RF0 cd_demo_sk->[ss_cdemo_sk] ------------------------PhysicalProject @@ -19,11 +19,11 @@ PhysicalResultSink --------------------------filter(((((customer_demographics.cd_marital_status = 'S') AND (customer_demographics.cd_education_status = 'Secondary')) OR ((customer_demographics.cd_marital_status = 'M') AND (customer_demographics.cd_education_status = '2 yr Degree'))) OR ((customer_demographics.cd_marital_status = 'D') AND (customer_demographics.cd_education_status = 'Advanced Degree'))) and cd_education_status IN ('2 yr Degree', 'Advanced Degree', 'Secondary') and cd_marital_status IN ('D', 'M', 'S')) ----------------------------PhysicalOlapScan[customer_demographics] --------------------PhysicalProject -----------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'GA', 'KS', 'MD', 'MN', 'NC', 'ND', 'NY', 'SD')) -------------------------PhysicalOlapScan[customer_address] +----------------------filter((date_dim.d_year = 2001)) +------------------------PhysicalOlapScan[date_dim] ----------------PhysicalProject -------------------filter((date_dim.d_year = 2001)) ---------------------PhysicalOlapScan[date_dim] +------------------filter((customer_address.ca_country = 'United States') and ca_state IN ('CO', 'GA', 'KS', 'MD', 'MN', 'NC', 'ND', 'NY', 'SD')) +--------------------PhysicalOlapScan[customer_address] ------------PhysicalProject --------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out index 889d5069ff97511..f420e8578c7d4bb 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query49.out @@ -1,107 +1,91 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalUnion +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((wr.wr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((ws.ws_item_sk = wr.wr_item_sk) and (ws.ws_order_number = wr.wr_order_number)) otherCondition=() build RFs:RF1 ws_order_number->[wr_order_number];RF2 ws_item_sk->[wr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[web_returns] apply RFs: RF1 RF2 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ws.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((ws.ws_net_paid > 0.00) and (ws.ws_net_profit > 1.00) and (ws.ws_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cr.cr_return_amount > 10000.00)) +------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((cs.cs_item_sk = cr.cr_item_sk) and (cs.cs_order_number = cr.cr_order_number)) otherCondition=() build RFs:RF4 cs_order_number->[cr_order_number];RF5 cs_item_sk->[cr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[catalog_returns] apply RFs: RF4 RF5 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((cs.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((cs.cs_net_paid > 0.00) and (cs.cs_net_profit > 1.00) and (cs.cs_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------PhysicalTopN[MERGE_SORT] ---------------------PhysicalDistribute[DistributionSpecGather] -----------------------PhysicalTopN[LOCAL_SORT] -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------------------------PhysicalWindow +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute[DistributionSpecExecutionAny] +----------------------PhysicalProject +------------------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +--------------------------PhysicalWindow +----------------------------PhysicalQuickSort[LOCAL_SORT] +------------------------------PhysicalWindow +--------------------------------PhysicalQuickSort[MERGE_SORT] +----------------------------------PhysicalDistribute[DistributionSpecGather] ------------------------------------PhysicalQuickSort[LOCAL_SORT] ---------------------------------------PhysicalWindow -----------------------------------------PhysicalQuickSort[MERGE_SORT] -------------------------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------PhysicalProject +----------------------------------------hashAgg[GLOBAL] +------------------------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((sr.sr_return_amt > 10000.00)) +------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 +--------------------------------------------------PhysicalProject +----------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] +------------------------------------------------------PhysicalProject +--------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) +----------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN colocated] hashCondition=((sts.ss_item_sk = sr.sr_item_sk) and (sts.ss_ticket_number = sr.sr_ticket_number)) otherCondition=() build RFs:RF7 ss_ticket_number->[sr_ticket_number];RF8 ss_item_sk->[sr_item_sk] -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF7 RF8 -----------------------------------------------------------PhysicalProject -------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((sts.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF6 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((sts.ss_net_paid > 0.00) and (sts.ss_net_profit > 1.00) and (sts.ss_quantity > 0)) -------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 ---------------------------------------------------------------PhysicalProject -----------------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) -------------------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------------filter((date_dim.d_moy = 11) and (date_dim.d_year = 1998)) +----------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query50.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query50.out index f5c3f38463d42ca..b45ce616bee1d2a 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query50.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query50.out @@ -1,29 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_store_sk = store.s_store_sk)) otherCondition=() build RFs:RF5 s_store_sk->[ss_store_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = d1.d_date_sk)) otherCondition=() build RFs:RF4 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------hashJoin[INNER_JOIN colocated] hashCondition=((store_sales.ss_customer_sk = store_returns.sr_customer_sk) and (store_sales.ss_item_sk = store_returns.sr_item_sk) and (store_sales.ss_ticket_number = store_returns.sr_ticket_number)) otherCondition=() build RFs:RF1 sr_ticket_number->[ss_ticket_number];RF2 sr_item_sk->[ss_item_sk];RF3 sr_customer_sk->[ss_customer_sk] ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +--------------------------------PhysicalOlapScan[store_sales] apply RFs: RF1 RF2 RF3 RF4 RF5 ------------------------------PhysicalProject ---------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_returns.sr_returned_date_sk = d2.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[sr_returned_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_returns] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((d2.d_moy = 8) and (d2.d_year = 2001)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[store] +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query54.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query54.out index 397a41b34c4e60b..2c1e5cdc05399ca 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query54.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query54.out @@ -23,7 +23,7 @@ PhysicalResultSink ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF6 RF7 ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_county->[ca_county];RF5 s_state->[ca_state] +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_county = store.s_county) and (customer_address.ca_state = store.s_state)) otherCondition=() build RFs:RF4 s_state->[ca_state];RF5 s_county->[ca_county] --------------------------------------------PhysicalProject ----------------------------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((my_customers.c_current_addr_sk = customer_address.ca_address_sk)) otherCondition=() build RFs:RF3 c_current_addr_sk->[ca_address_sk] ------------------------------------------------PhysicalProject diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query6.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query6.out index 43c8732d7f8553f..18dbe974af8c0c4 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query6.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query6.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------filter((cnt >= 10)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute[DistributionSpecHash] -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] apply RFs: RF4 ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 ---------------------------------------PhysicalAssertNumRows -----------------------------------------PhysicalDistribute[DistributionSpecGather] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) -----------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] apply RFs: RF0 -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute[DistributionSpecHash] ---------------------------------------hashAgg[LOCAL] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------filter((cnt >= 10)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute[DistributionSpecHash] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((a.ca_address_sk = c.c_current_addr_sk)) otherCondition=() build RFs:RF5 c_current_addr_sk->[ca_address_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] apply RFs: RF5 +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_customer_sk = s.ss_customer_sk)) otherCondition=() build RFs:RF4 ss_customer_sk->[c_customer_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] apply RFs: RF4 +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_item_sk = i.i_item_sk)) otherCondition=() build RFs:RF3 i_item_sk->[ss_item_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((s.ss_sold_date_sk = d.d_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d.d_month_seq = date_dim.d_month_seq)) otherCondition=() build RFs:RF1 d_month_seq->[d_month_seq] ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[date_dim] apply RFs: RF1 +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute[DistributionSpecGather] +--------------------------------------------hashAgg[GLOBAL] +----------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_moy = 3) and (date_dim.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((j.i_category = i.i_category)) otherCondition=((cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(cast(i_current_price as DECIMALV3(9, 4)))))) build RFs:RF0 i_category->[i_category] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[item] apply RFs: RF0 +------------------------------------hashAgg[GLOBAL] +--------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------hashAgg[LOCAL] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query66.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query66.out index 924459179feee99..7a96568921c1605 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query66.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query66.out @@ -4,59 +4,60 @@ PhysicalResultSink --PhysicalTopN[MERGE_SORT] ----PhysicalDistribute[DistributionSpecGather] ------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute[DistributionSpecHash] -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 -------------------------------------------PhysicalProject ---------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) -----------------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_year = 2001)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) ---------------------------------------PhysicalOlapScan[time_dim] -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[warehouse] +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute[DistributionSpecHash] +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF3 w_warehouse_sk->[ws_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF2 t_time_sk->[ws_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF1 d_date_sk->[ws_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF0 sm_ship_mode_sk->[ws_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 RF1 RF2 RF3 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk)) otherCondition=() build RFs:RF7 w_warehouse_sk->[cs_warehouse_sk] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_time_sk = time_dim.t_time_sk)) otherCondition=() build RFs:RF6 t_time_sk->[cs_sold_time_sk] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[cs_sold_date_sk] +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk)) otherCondition=() build RFs:RF4 sm_ship_mode_sk->[cs_ship_mode_sk] +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF4 RF5 RF6 RF7 +--------------------------------------------PhysicalProject +----------------------------------------------filter(sm_carrier IN ('BOXBUNDLES', 'ORIENTAL')) +------------------------------------------------PhysicalOlapScan[ship_mode] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_year = 2001)) +--------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((cast(t_time as BIGINT) <= 71770) and (time_dim.t_time >= 42970)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query69.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query69.out index 31101f12eab21ab..8ee7c8a7e0aa7ef 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query69.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query69.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute[DistributionSpecHash] ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 -------------------------PhysicalProject ---------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) -----------------------------PhysicalOlapScan[date_dim] ---------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN shuffleBucket] hashCondition=((c.c_customer_sk = store_sales.ss_customer_sk)) otherCondition=() build RFs:RF6 c_customer_sk->[ss_customer_sk] ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF5 d_date_sk->[ss_sold_date_sk] --------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 +----------------------------PhysicalOlapScan[store_sales] apply RFs: RF5 RF6 --------------------------PhysicalProject ----------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) ------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 ---------------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +----------------------hashJoin[RIGHT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = catalog_sales.cs_ship_customer_sk)) otherCondition=() build RFs:RF4 c_customer_sk->[cs_ship_customer_sk] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF3 RF4 ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[customer] apply RFs: RF1 ---------------------------------PhysicalProject -----------------------------------filter(ca_state IN ('IL', 'ME', 'TX')) -------------------------------------PhysicalOlapScan[customer_address] +------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_demographics.cd_demo_sk = c.c_current_cdemo_sk)) otherCondition=() build RFs:RF2 c_current_cdemo_sk->[cd_demo_sk] ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[customer_demographics] apply RFs: RF2 +----------------------------hashJoin[LEFT_ANTI_JOIN shuffle] hashCondition=((c.c_customer_sk = web_sales.ws_bill_customer_sk)) otherCondition=() +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((c.c_current_addr_sk = ca.ca_address_sk)) otherCondition=() build RFs:RF1 ca_address_sk->[c_current_addr_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[customer] apply RFs: RF1 +----------------------------------PhysicalProject +------------------------------------filter(ca_state IN ('IL', 'ME', 'TX')) +--------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((web_sales.ws_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ws_sold_date_sk] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] apply RFs: RF0 +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_moy <= 3) and (date_dim.d_moy >= 1) and (date_dim.d_year = 2002)) +--------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query70.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query70.out index ec1bdd0e99afb61..f46d7ff3d3e03a7 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query70.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query70.out @@ -13,32 +13,39 @@ PhysicalResultSink --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute[DistributionSpecHash] ------------------------hashAgg[LOCAL] ---------------------------PhysicalRepeat -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF4 s_store_sk->[ss_store_sk] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[ss_sold_date_sk] +--------------------------PhysicalProject +----------------------------PhysicalRepeat +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF3 s_store_sk->[ss_store_sk] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((d1.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF2 d_date_sk->[ss_sold_date_sk] +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF2 RF3 +--------------------------------------PhysicalProject +----------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------hashJoin[RIGHT_SEMI_JOIN shuffle] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF3 RF4 +--------------------------------------filter((ranking <= 5)) +----------------------------------------PhysicalWindow +------------------------------------------PhysicalQuickSort[LOCAL_SORT] +--------------------------------------------PhysicalDistribute[DistributionSpecHash] +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute[DistributionSpecHash] +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 +----------------------------------------------------------------PhysicalProject +------------------------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) +--------------------------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store] ------------------------------------PhysicalProject ---------------------------------------filter((d1.d_month_seq <= 1231) and (d1.d_month_seq >= 1220)) -----------------------------------------PhysicalOlapScan[date_dim] ---------------------------------hashJoin[LEFT_SEMI_JOIN broadcast] hashCondition=((store.s_state = tmp1.s_state)) otherCondition=() build RFs:RF2 s_state->[s_state] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] apply RFs: RF2 -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] ---------------------------------------PhysicalDistribute[DistributionSpecHash] -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store.s_store_sk = store_sales.ss_store_sk)) otherCondition=() build RFs:RF1 s_store_sk->[ss_store_sk] -----------------------------------------------PhysicalProject -------------------------------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((date_dim.d_date_sk = store_sales.ss_sold_date_sk)) otherCondition=() build RFs:RF0 d_date_sk->[ss_sold_date_sk] ---------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF1 ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_month_seq <= 1231) and (date_dim.d_month_seq >= 1220)) -------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store] +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out index 473b9fded857150..d40e04f285ad81e 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query76.out @@ -1,40 +1,41 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalResultSink ---PhysicalTopN[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalTopN[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +--PhysicalProject +----PhysicalTopN[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalTopN[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalUnion -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] -----------------------------PhysicalProject -------------------------------filter(ss_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] apply RFs: RF1 -----------------------------PhysicalProject -------------------------------filter(ws_promo_sk IS NULL) ---------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 -----------------------PhysicalDistribute[DistributionSpecExecutionAny] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] -----------------------------PhysicalProject -------------------------------filter(cs_bill_customer_sk IS NULL) ---------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalProject ---------------------PhysicalOlapScan[date_dim] +--------------------hashJoin[INNER_JOIN broadcast] hashCondition=((ss_sold_date_sk = date_dim.d_date_sk)) otherCondition=() build RFs:RF3 d_date_sk->[cs_sold_date_sk,ss_sold_date_sk,ws_sold_date_sk] +----------------------PhysicalUnion +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((store_sales.ss_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF0 i_item_sk->[ss_item_sk] +------------------------------PhysicalProject +--------------------------------filter(ss_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[store_sales] apply RFs: RF0 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((web_sales.ws_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF1 ws_item_sk->[i_item_sk] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] apply RFs: RF1 +------------------------------PhysicalProject +--------------------------------filter(ws_promo_sk IS NULL) +----------------------------------PhysicalOlapScan[web_sales] apply RFs: RF3 +------------------------PhysicalDistribute[DistributionSpecHash] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN broadcast] hashCondition=((catalog_sales.cs_item_sk = item.i_item_sk)) otherCondition=() build RFs:RF2 i_item_sk->[cs_item_sk] +------------------------------PhysicalProject +--------------------------------filter(cs_bill_customer_sk IS NULL) +----------------------------------PhysicalOlapScan[catalog_sales] apply RFs: RF2 RF3 +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] +----------------------PhysicalProject +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query81.out b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query81.out index b3b6627dd0716da..3eddfb269717eaf 100644 --- a/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query81.out +++ b/regression-test/data/new_shapes_p0/tpcds_sf1000/shape/query81.out @@ -18,24 +18,25 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] --PhysicalResultSink -----PhysicalTopN[MERGE_SORT] -------PhysicalDistribute[DistributionSpecGather] ---------PhysicalTopN[LOCAL_SORT] -----------PhysicalProject -------------hashJoin[INNER_JOIN broadcast] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF3 c_customer_sk->[ctr_customer_sk] -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF3 RF4 -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF2 ca_address_sk->[c_current_addr_sk] -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] apply RFs: RF2 -----------------------PhysicalProject -------------------------filter((customer_address.ca_state = 'TN')) ---------------------------PhysicalOlapScan[customer_address] ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute[DistributionSpecHash] -------------------hashAgg[LOCAL] ---------------------PhysicalDistribute[DistributionSpecExecutionAny] -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----PhysicalProject +------PhysicalTopN[MERGE_SORT] +--------PhysicalDistribute[DistributionSpecGather] +----------PhysicalTopN[LOCAL_SORT] +------------PhysicalProject +--------------hashJoin[INNER_JOIN bucketShuffle] hashCondition=((ctr1.ctr_state = ctr2.ctr_state)) otherCondition=((cast(ctr_total_return as DOUBLE) > cast((avg(cast(ctr_total_return as DECIMALV3(38, 4))) * 1.2) as DOUBLE))) build RFs:RF4 ctr_state->[ctr_state] +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute[DistributionSpecHash] +--------------------hashAgg[LOCAL] +----------------------PhysicalDistribute[DistributionSpecExecutionAny] +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) apply RFs: RF4 +----------------PhysicalProject +------------------hashJoin[INNER_JOIN broadcast] hashCondition=((customer_address.ca_address_sk = customer.c_current_addr_sk)) otherCondition=() build RFs:RF3 ca_address_sk->[c_current_addr_sk] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN shuffle] hashCondition=((ctr1.ctr_customer_sk = customer.c_customer_sk)) otherCondition=() build RFs:RF2 ctr_customer_sk->[c_customer_sk] +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] apply RFs: RF2 RF3 +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalProject +----------------------filter((customer_address.ca_state = 'TN')) +------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q12.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q12.out index 8df830dd428e58a..99ea02d4359a948 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q12.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q16.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q16.out index 7b04caaf3e087a0..95c7b4b338279ca 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q16.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q16.out @@ -1,21 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] -------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -----------------------PhysicalOlapScan[part] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +----------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +------------------------PhysicalProject +--------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +------------------------PhysicalProject +--------------------------filter((s_comment like '%Customer%Complaints%')) +----------------------------PhysicalOlapScan[supplier] +----------------------PhysicalProject +------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q22.out b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q22.out index 63d82280b35b168..c3d504ca32c73ca 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q22.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/nostats_rf_prune/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q12.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q12.out index 8df830dd428e58a..99ea02d4359a948 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q12.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q16.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q16.out index f04b0bc766338b7..3158a8c782c15f0 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q16.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q16.out @@ -1,21 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() -----------------PhysicalProject -------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -------------------------PhysicalOlapScan[part] -----------------PhysicalProject -------------------filter((s_comment like '%Customer%Complaints%')) ---------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +------------------------------PhysicalOlapScan[part] +----------------------PhysicalProject +------------------------filter((s_comment like '%Customer%Complaints%')) +--------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q22.out b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q22.out index 63d82280b35b168..c3d504ca32c73ca 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q22.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/rf_prune/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q12.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q12.out index 8df830dd428e58a..99ea02d4359a948 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q12.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q16.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q16.out index f04b0bc766338b7..3158a8c782c15f0 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q16.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q16.out @@ -1,21 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() -----------------PhysicalProject -------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -------------------------PhysicalOlapScan[part] -----------------PhysicalProject -------------------filter((s_comment like '%Customer%Complaints%')) ---------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +--------------------------PhysicalProject +----------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +------------------------------PhysicalOlapScan[part] +----------------------PhysicalProject +------------------------filter((s_comment like '%Customer%Complaints%')) +--------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q22.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q22.out index 63d82280b35b168..c3d504ca32c73ca 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q22.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q12.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q12.out index 8df830dd428e58a..99ea02d4359a948 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q12.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q12.out @@ -1,17 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) -----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((orders.o_orderkey = lineitem.l_orderkey)) otherCondition=() build RFs:RF0 l_orderkey->[o_orderkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 +----------------------PhysicalProject +------------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate) and (lineitem.l_receiptdate < '1995-01-01') and (lineitem.l_receiptdate >= '1994-01-01') and (lineitem.l_shipdate < '1995-01-01') and (lineitem.l_shipdate < lineitem.l_commitdate) and l_shipmode IN ('MAIL', 'SHIP')) +--------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q16.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q16.out index 7b04caaf3e087a0..95c7b4b338279ca 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q16.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q16.out @@ -1,21 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] -------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] apply RFs: RF0 ---------------------PhysicalProject -----------------------filter((s_comment like '%Customer%Complaints%')) -------------------------PhysicalOlapScan[supplier] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) -----------------------PhysicalOlapScan[part] +--------------------hashJoin[INNER_JOIN colocated] hashCondition=((part.p_partkey = partsupp.ps_partkey)) otherCondition=() build RFs:RF0 p_partkey->[ps_partkey] +----------------------hashJoin[LEFT_ANTI_JOIN broadcast] hashCondition=((partsupp.ps_suppkey = supplier.s_suppkey)) otherCondition=() +------------------------PhysicalProject +--------------------------PhysicalOlapScan[partsupp] apply RFs: RF0 +------------------------PhysicalProject +--------------------------filter((s_comment like '%Customer%Complaints%')) +----------------------------PhysicalOlapScan[supplier] +----------------------PhysicalProject +------------------------filter(( not (p_brand = 'Brand#45')) and ( not (p_type like 'MEDIUM POLISHED%')) and p_size IN (14, 19, 23, 3, 36, 45, 49, 9)) +--------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q22.out b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q22.out index 63d82280b35b168..c3d504ca32c73ca 100644 --- a/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q22.out +++ b/regression-test/data/new_shapes_p0/tpch_sf1000/shape_no_stats/q22.out @@ -1,25 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- PhysicalResultSink ---PhysicalQuickSort[MERGE_SORT] -----PhysicalDistribute[DistributionSpecGather] -------PhysicalQuickSort[LOCAL_SORT] ---------hashAgg[GLOBAL] -----------PhysicalDistribute[DistributionSpecHash] -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +--PhysicalProject +----PhysicalQuickSort[MERGE_SORT] +------PhysicalDistribute[DistributionSpecGather] +--------PhysicalQuickSort[LOCAL_SORT] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute[DistributionSpecHash] +----------------hashAgg[LOCAL] ------------------PhysicalProject ---------------------PhysicalOlapScan[orders] apply RFs: RF0 -------------------PhysicalProject ---------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------hashJoin[RIGHT_ANTI_JOIN shuffleBucket] hashCondition=((orders.o_custkey = customer.c_custkey)) otherCondition=() build RFs:RF0 c_custkey->[o_custkey] +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] apply RFs: RF0 ----------------------PhysicalProject -------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------PhysicalOlapScan[customer] -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute[DistributionSpecGather] ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) ---------------------------------PhysicalOlapScan[customer] +------------------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(cast(c_acctbal as DECIMALV3(17, 4)))) +--------------------------PhysicalProject +----------------------------filter(substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------PhysicalOlapScan[customer] +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute[DistributionSpecGather] +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------filter((customer.c_acctbal > 0.00) and substring(c_phone, 1, 2) IN ('13', '17', '18', '23', '29', '30', '31')) +------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/suites/nereids_p0/compress_materialize/compress_materialize.groovy b/regression-test/suites/nereids_p0/compress_materialize/compress_materialize.groovy new file mode 100644 index 000000000000000..3360426e3344fec --- /dev/null +++ b/regression-test/suites/nereids_p0/compress_materialize/compress_materialize.groovy @@ -0,0 +1,196 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +suite("compress_materialize") { + sql """ + drop table if exists compress; + CREATE TABLE `compress` ( + `k` varchar(5) NOT NULL, + `v` int NOT NULL + ) ENGINE=OLAP + duplicate KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS AUTO + PROPERTIES ( + "replication_num" = "1" + ); + + + insert into compress values ("aaaaaa", 1), ("aaaaaa", 2), ("bbbbb", 3), ("bbbbb", 4), ("bbbbb", 5); + + + drop table if exists cmt2; + CREATE TABLE `cmt2` ( + `k2` varchar(5) NOT NULL, + `v2` int NOT NULL + ) ENGINE=OLAP + duplicate KEY(`k2`) + DISTRIBUTED BY random + PROPERTIES ( + "replication_num" = "1" + ); + + insert into cmt2 values ("aaaa", 1), ("b", 3); + insert into cmt2 values("123456", 123456); + """ + + explain{ + sql (""" + select k from compress group by k; + """) + contains("encode_as_bigint") + } + order_qt_agg_exec "select k from compress group by k;" + + explain{ + sql (""" + select k, substring(k, 1), sum(v) from compress group by k; + """) + contains("encode_as_bigint(k)") + } + order_qt_output_contains_gpk "select k, substring(k, 1) from compress group by k;" + + order_qt_expr """ select substring(k,1,3) from compress group by substring(k,1,3);""" + explain{ + sql "select substring(k,1,3) from compress group by substring(k,1,3);" + contains("encode_as_int(substring(k, 1, 3))") + } + + explain { + sql("select sum(v) from compress group by substring(k, 1, 3);") + contains("group by: encode_as_int(substring(k, 1, 3))") + } + + explain { + sql("select sum(v) from compress group by substring(k, 1, 4);") + contains("group by: encode_as_bigint(substring(k, 1, 4))") + } + + order_qt_encodeexpr "select sum(v) from compress group by substring(k, 1, 3);" + + // TODO: RF targets on compressed_materialze column is broken + // // verify that compressed materialization do not block runtime filter generation + // sql """ + // set disable_join_reorder=true; + // set runtime_filter_mode = GLOBAL; + // set runtime_filter_type=2; + // set enable_runtime_filter_prune=false; + // """ + + // qt_join """ + // explain shape plan + // select * + // from ( + // select k from compress group by k + // ) T join[broadcast] cmt2 on T.k = cmt2.k2; + // """ + + + sql """ + drop table if exists compressInt; + CREATE TABLE `compressInt` ( + `k` varchar(3) NOT NULL, + `v` int NOT NULL + ) ENGINE=OLAP + duplicate KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS AUTO + PROPERTIES ( + "replication_num" = "1" + ); + + + insert into compressInt values ("a", 1), ("aa", 2), ("bb", 3), ("b", 4), ("b", 5); + """ + explain{ + sql "select k from compressInt group by k" + contains("encode_as_int") + } + + sql """ + drop table if exists compressLargeInt; + CREATE TABLE `compressLargeInt` ( + `k` varchar(10) NOT NULL, + `v` int NOT NULL + ) ENGINE=OLAP + duplicate KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS AUTO + PROPERTIES ( + "replication_num" = "1" + ); + + + insert into compressLargeInt values ("a", 1), ("aa", 2), ("bb", 3), ("b", 4), ("b", 5); + """ + explain{ + sql "select k from compressLargeInt group by k" + contains("group by: encode_as_largeint(k)") + } + + + sql """ + drop table if exists notcompress; + CREATE TABLE `notcompress` ( + `k` varchar(16) NOT NULL, + `v` int NOT NULL + ) ENGINE=OLAP + duplicate KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS AUTO + PROPERTIES ( + "replication_num" = "1" + ); + + + insert into notcompress values ("a", 1), ("aa", 2), ("bb", 3), ("b", 4), ("b", 5); + """ + explain{ + sql "select k from notcompress group by k" + notContains("encode_as_") + } + + sql """ + drop table if exists compressSort; + CREATE TABLE `compressSort` ( + `k` varchar(3) NULL, + `v` int NOT NULL + ) ENGINE=OLAP + duplicate KEY(`k`) + DISTRIBUTED BY HASH(`k`) BUCKETS AUTO + PROPERTIES ( + "replication_num" = "1" + ); + + + insert into compressSort values ("a", 1), ("aa", 2), ("bb", 3), ("b", 4), ("b", 5); + insert into compressSort(v) values (6); + insert into compressSort values ("",7), ("中", 8), ("国", 9); + """ + explain { + sql "select v from compressSort order by k" + contains("order by: encode_as_int(k)") +// expect plan fragment: +// 1:VSORT(140) | +// order by: encode_as_int(k)[#5] ASC | +// algorithm: full sort | +// offset: 0 | +// distribute expr lists: + } + qt_sort "select * from compressSort order by k asc, v"; + qt_sort "select * from compressSort order by k desc, v"; + qt_sort "select * from compressSort order by k desc nulls last"; + qt_sort "select * from compressSort order by k desc nulls last, v limit 3"; + +} +