diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java index ad3d65b8dba4fa..bf74eceee387c5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java @@ -370,6 +370,7 @@ private PhysicalPlan chooseBestPlan(Group rootGroup, PhysicalProperties physical Plan plan = groupExpression.getPlan().withChildren(planChildren); if (!(plan instanceof PhysicalPlan)) { + // TODO need add some log throw new AnalysisException("Result plan must be PhysicalPlan"); } // add groupExpression to plan so that we could print group id in plan.treeString() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java index d085baf02f92a3..bdf34f43b7418b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/ExpressionTranslator.java @@ -75,7 +75,6 @@ import org.apache.doris.nereids.trees.expressions.WhenClause; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNotNullable; import org.apache.doris.nereids.trees.expressions.functions.AlwaysNullable; -import org.apache.doris.nereids.trees.expressions.functions.BoundFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; @@ -365,14 +364,11 @@ public Expr visitWindowFunction(WindowFunction function, PlanTranslatorContext c @Override public Expr visitScalarFunction(ScalarFunction function, PlanTranslatorContext context) { - List nereidsArguments = adaptFunctionArgumentsForBackends(function); - - List arguments = nereidsArguments - .stream() + List arguments = function.getArguments().stream() .map(arg -> arg.accept(this, context)) .collect(Collectors.toList()); - List argTypes = nereidsArguments.stream() + List argTypes = function.getArguments().stream() .map(Expression::getDataType) .map(AbstractDataType::toCatalogDataType) .collect(Collectors.toList()); @@ -594,12 +590,4 @@ private static org.apache.doris.analysis.AssertNumRowsElement.Assertion translat throw new AnalysisException("UnSupported type: " + assertion); } } - - /** - * some special arguments not need exists in the nereids, and backends need it, so we must add the - * special arguments for backends, e.g. the json data type string in the json_object function. - */ - private List adaptFunctionArgumentsForBackends(BoundFunction function) { - return function.getArguments(); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 14dbb80e145e9d..1f504af02fb0f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java @@ -38,7 +38,6 @@ import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.analysis.TupleId; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Env; import org.apache.doris.catalog.Function.NullableMode; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; @@ -49,23 +48,24 @@ import org.apache.doris.catalog.external.IcebergExternalTable; import org.apache.doris.catalog.external.JdbcExternalTable; import org.apache.doris.catalog.external.PaimonExternalTable; -import org.apache.doris.common.Pair; import org.apache.doris.common.UserException; import org.apache.doris.common.util.Util; import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.properties.DistributionSpec; import org.apache.doris.nereids.properties.DistributionSpecAny; +import org.apache.doris.nereids.properties.DistributionSpecExecutionAny; import org.apache.doris.nereids.properties.DistributionSpecGather; import org.apache.doris.nereids.properties.DistributionSpecHash; -import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.properties.DistributionSpecReplicated; +import org.apache.doris.nereids.properties.DistributionSpecStorageAny; +import org.apache.doris.nereids.properties.DistributionSpecStorageGather; import org.apache.doris.nereids.properties.OrderKey; -import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow.WindowFrameGroup; import org.apache.doris.nereids.stats.StatsErrorEstimator; +import org.apache.doris.nereids.trees.UnaryNode; import org.apache.doris.nereids.trees.expressions.AggregateExpression; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.CTEId; -import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; @@ -120,10 +120,8 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; import org.apache.doris.nereids.trees.plans.physical.RuntimeFilter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; -import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.JoinUtils; -import org.apache.doris.nereids.util.TypeCoercionUtils; import org.apache.doris.nereids.util.Utils; import org.apache.doris.planner.AggregationNode; import org.apache.doris.planner.AnalyticEvalNode; @@ -161,6 +159,7 @@ import org.apache.doris.planner.external.iceberg.IcebergScanNode; import org.apache.doris.planner.external.paimon.PaimonScanNode; import org.apache.doris.qe.ConnectContext; +import org.apache.doris.system.SystemInfoService; import org.apache.doris.tablefunction.TableValuedFunctionIf; import org.apache.doris.thrift.TColumn; import org.apache.doris.thrift.TFetchOption; @@ -169,7 +168,6 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableList.Builder; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; import com.google.common.collect.Maps; @@ -180,12 +178,10 @@ import java.util.ArrayList; import java.util.Collection; import java.util.Collections; -import java.util.HashMap; import java.util.HashSet; import java.util.List; import java.util.Map; import java.util.Objects; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; import java.util.stream.Stream; @@ -217,54 +213,6 @@ public PhysicalPlanTranslator(PlanTranslatorContext context, StatsErrorEstimator this.statsErrorEstimator = statsErrorEstimator; } - // We use two phase read to optimize sql like: select * from tbl [where xxx = ???] [order by column1] [limit n] - // in the first phase, we add an extra column `RowId` to Block, and sort blocks in TopN nodes - // in the second phase, we have n rows, we do a fetch rpc to get all rowids data for the n rows - // and reconconstruct the final block - private void setResultSinkFetchOptionIfNeed() { - boolean needFetch = false; - // Only single olap table should be fetched - OlapTable fetchOlapTable = null; - OlapScanNode scanNode = null; - for (PlanFragment fragment : context.getPlanFragments()) { - PlanNode node = fragment.getPlanRoot(); - PlanNode parent = null; - // OlapScanNode is the last node. - // So, just get the last two node and check if they are SortNode and OlapScan. - while (node.getChildren().size() != 0) { - parent = node; - node = node.getChildren().get(0); - } - - // case1: general topn optimized query - if ((node instanceof OlapScanNode) && (parent instanceof SortNode)) { - SortNode sortNode = (SortNode) parent; - scanNode = (OlapScanNode) node; - if (sortNode.getUseTwoPhaseReadOpt()) { - needFetch = true; - fetchOlapTable = scanNode.getOlapTable(); - break; - } - } - } - for (PlanFragment fragment : context.getPlanFragments()) { - if (needFetch && fragment.getSink() instanceof ResultSink) { - TFetchOption fetchOption = new TFetchOption(); - fetchOption.setFetchRowStore(fetchOlapTable.storeRowColumn()); - fetchOption.setUseTwoPhaseFetch(true); - fetchOption.setNodesInfo(Env.getCurrentSystemInfo().createAliveNodesInfo()); - if (!fetchOlapTable.storeRowColumn()) { - // Set column desc for each column - List columnsDesc = new ArrayList(); - scanNode.getColumnDesc(columnsDesc, null, null); - fetchOption.setColumnDesc(columnsDesc); - } - ((ResultSink) fragment.getSink()).setFetchOption(fetchOption); - break; - } - } - } - /** * Translate Nereids Physical Plan tree to Stale Planner PlanFragment tree. * @@ -273,27 +221,9 @@ private void setResultSinkFetchOptionIfNeed() { */ public PlanFragment translatePlan(PhysicalPlan physicalPlan) { PlanFragment rootFragment = physicalPlan.accept(this, context); - if (physicalPlan instanceof PhysicalDistribute) { - PhysicalDistribute distribute = (PhysicalDistribute) physicalPlan; - DataPartition dataPartition; - if (distribute.getDistributionSpec().equals(PhysicalProperties.GATHER.getDistributionSpec())) { - dataPartition = DataPartition.UNPARTITIONED; - } else { - throw new AnalysisException("Unsupported PhysicalDistribute in the root plan: " + distribute); - } - ExchangeNode exchangeNode = (ExchangeNode) rootFragment.getPlanRoot(); - PlanFragment currentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, dataPartition); - rootFragment.setOutputPartition(dataPartition); - rootFragment.setPlanRoot(exchangeNode.getChild(0)); - rootFragment.setDestination(exchangeNode); - context.addPlanFragment(currentFragment); - rootFragment = currentFragment; - } - - if (!(physicalPlan instanceof PhysicalOlapTableSink) && isFragmentPartitioned(rootFragment)) { - rootFragment = exchangeToMergeFragment(rootFragment, context); - } + // TODO: why we need if? we should always set output expr? + // OlapSink? maybe OlapSink should not set output exprs by it self if (rootFragment.getOutputExprs() == null) { List outputExprs = Lists.newArrayList(); physicalPlan.getOutput().stream().map(Slot::getExprId) @@ -305,10 +235,63 @@ public PlanFragment translatePlan(PhysicalPlan physicalPlan) { } setResultSinkFetchOptionIfNeed(); Collections.reverse(context.getPlanFragments()); + // TODO: maybe we need to trans nullable directly? and then we could remove call computeMemLayout context.getDescTable().computeMemLayout(); return rootFragment; } + /* ******************************************************************************************** + * distribute node + * ******************************************************************************************** */ + + @Override + public PlanFragment visitPhysicalDistribute(PhysicalDistribute distribute, + PlanTranslatorContext context) { + PlanFragment childFragment = distribute.child().accept(this, context); + // TODO: why need set streaming here? should remove this. + if (childFragment.getPlanRoot() instanceof AggregationNode + && distribute.child() instanceof PhysicalHashAggregate + && context.getFirstAggregateInFragment(childFragment) == distribute.child()) { + PhysicalHashAggregate hashAggregate = (PhysicalHashAggregate) distribute.child(); + if (hashAggregate.getAggPhase() == AggPhase.LOCAL + && hashAggregate.getAggMode() == AggMode.INPUT_TO_BUFFER) { + AggregationNode aggregationNode = (AggregationNode) childFragment.getPlanRoot(); + aggregationNode.setUseStreamingPreagg(hashAggregate.isMaybeUsingStream()); + } + } + + ExchangeNode exchangeNode = new ExchangeNode(context.nextPlanNodeId(), childFragment.getPlanRoot()); + updateLegacyPlanIdToPhysicalPlan(exchangeNode, distribute); + exchangeNode.setNumInstances(childFragment.getPlanRoot().getNumInstances()); + if (distribute.getDistributionSpec() instanceof DistributionSpecGather) { + // gather to one instance + exchangeNode.setNumInstances(1); + } + + List validOutputIds = distribute.getOutputExprIds(); + if (distribute.child() instanceof PhysicalHashAggregate) { + // we must add group by keys to output list, + // otherwise we could not process aggregate's output without group by keys + List keys = ((PhysicalHashAggregate) distribute.child()).getGroupByExpressions().stream() + .filter(SlotReference.class::isInstance) + .map(SlotReference.class::cast) + .map(SlotReference::getExprId) + .collect(Collectors.toList()); + keys.addAll(validOutputIds); + validOutputIds = keys; + } + DataPartition dataPartition = toDataPartition(distribute.getDistributionSpec(), validOutputIds, context); + PlanFragment parentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, dataPartition); + childFragment.setDestination(exchangeNode); + childFragment.setOutputPartition(dataPartition); + context.addPlanFragment(parentFragment); + return parentFragment; + } + + /* ******************************************************************************************** + * sink Node, in lexicographical order + * ******************************************************************************************** */ + @Override public PlanFragment visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, PlanTranslatorContext context) { @@ -331,206 +314,75 @@ public PlanFragment visitPhysicalOlapTableSink(PhysicalOlapTableSink aggregate, - PlanTranslatorContext context) { - - PlanFragment inputPlanFragment = aggregate.child(0).accept(this, context); - - List groupByExpressionList = aggregate.getGroupByExpressions(); - List outputExpressionList = aggregate.getOutputExpressions(); + /* ******************************************************************************************** + * scan Node, in lexicographical order + * ******************************************************************************************** */ - // 1. generate slot reference for each group expression - List groupSlotList = collectGroupBySlots(groupByExpressionList, outputExpressionList); - ArrayList execGroupingExpressions = groupByExpressionList.stream() - .map(e -> ExpressionTranslator.translate(e, context)) - .collect(Collectors.toCollection(ArrayList::new)); - // 2. collect agg expressions and generate agg function to slot reference map - List aggFunctionOutput = Lists.newArrayList(); - List aggregateExpressionList = outputExpressionList.stream() - .filter(o -> o.anyMatch(AggregateExpression.class::isInstance)) - .peek(o -> aggFunctionOutput.add(o.toSlot())) - .map(o -> o.>collect(AggregateExpression.class::isInstance)) - .flatMap(Set::stream) - .collect(Collectors.toList()); - ArrayList execAggregateFunctions = aggregateExpressionList.stream() - .map(aggregateFunction -> (FunctionCallExpr) ExpressionTranslator.translate(aggregateFunction, context)) - .collect(Collectors.toCollection(ArrayList::new)); + @Override + public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTranslatorContext context) { + List slots = fileScan.getOutput(); + ExternalTable table = fileScan.getTable(); + TupleDescriptor tupleDescriptor = generateTupleDesc(slots, table, context); - PlanFragment currentFragment; - if (inputPlanFragment.getPlanRoot() instanceof ExchangeNode - && aggregate.child() instanceof PhysicalDistribute) { - //the exchange node is generated in two cases: - // 1. some nodes (e.g. sort node) need to gather data from multiple instances, and hence their gather phase - // need an exchange node. For this type of exchange, their data partition is un_partitioned, do not - // create a new plan fragment. - // 2. PhysicalDistribute node is translated to exchange node. PhysicalDistribute node means we need to - // shuffle data, and we have to create a new plan fragment. - ExchangeNode exchangeNode = (ExchangeNode) inputPlanFragment.getPlanRoot(); - Optional> partitionExpressions = aggregate.getPartitionExpressions(); - PhysicalDistribute physicalDistribute = (PhysicalDistribute) aggregate.child(); - DataPartition dataPartition = toDataPartition(physicalDistribute, partitionExpressions, context).get(); - currentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, dataPartition); - inputPlanFragment.setOutputPartition(dataPartition); - inputPlanFragment.setPlanRoot(exchangeNode.getChild(0)); - inputPlanFragment.setDestination(exchangeNode); - context.addPlanFragment(currentFragment); + // TODO(cmy): determine the needCheckColumnPriv param + ScanNode scanNode; + if (table instanceof HMSExternalTable) { + switch (((HMSExternalTable) table).getDlaType()) { + case HUDI: + scanNode = new HudiScanNode(context.nextPlanNodeId(), tupleDescriptor, false); + break; + case ICEBERG: + scanNode = new IcebergScanNode(context.nextPlanNodeId(), tupleDescriptor, false); + break; + case HIVE: + scanNode = new HiveScanNode(context.nextPlanNodeId(), tupleDescriptor, false); + break; + default: + throw new RuntimeException("do not support DLA type " + ((HMSExternalTable) table).getDlaType()); + } + } else if (table instanceof IcebergExternalTable) { + scanNode = new IcebergScanNode(context.nextPlanNodeId(), tupleDescriptor, false); + } else if (table instanceof PaimonExternalTable) { + scanNode = new PaimonScanNode(context.nextPlanNodeId(), tupleDescriptor, false); } else { - currentFragment = inputPlanFragment; - } - - // 3. generate output tuple - List slotList = Lists.newArrayList(); - TupleDescriptor outputTupleDesc; - slotList.addAll(groupSlotList); - slotList.addAll(aggFunctionOutput); - outputTupleDesc = generateTupleDesc(slotList, null, context); - - List aggFunOutputIds = ImmutableList.of(); - if (!aggFunctionOutput.isEmpty()) { - aggFunOutputIds = outputTupleDesc - .getSlots() - .subList(groupSlotList.size(), outputTupleDesc.getSlots().size()) - .stream() - .map(slot -> slot.getId().asInt()) - .collect(ImmutableList.toImmutableList()); - } - boolean isPartial = aggregate.getAggregateParam().aggMode.productAggregateBuffer; - AggregateInfo aggInfo = AggregateInfo.create(execGroupingExpressions, execAggregateFunctions, - aggFunOutputIds, isPartial, outputTupleDesc, outputTupleDesc, aggregate.getAggPhase().toExec()); - AggregationNode aggregationNode = new AggregationNode(context.nextPlanNodeId(), - currentFragment.getPlanRoot(), aggInfo); - if (!aggregate.getAggMode().isFinalPhase) { - aggregationNode.unsetNeedsFinalize(); - } - PhysicalHashAggregate firstAggregateInFragment = context.getFirstAggregateInFragment(currentFragment); - - switch (aggregate.getAggPhase()) { - case LOCAL: - // we should set is useStreamingAgg when has exchange, - // so the `aggregationNode.setUseStreamingPreagg()` in the visitPhysicalDistribute - break; - case DISTINCT_LOCAL: - aggregationNode.setIntermediateTuple(); - break; - case GLOBAL: - case DISTINCT_GLOBAL: - break; - default: - throw new RuntimeException("Unsupported yet"); + throw new RuntimeException("do not support table type " + table.getType()); } - if (firstAggregateInFragment == null) { - context.setFirstAggregateInFragment(currentFragment, aggregate); - } - // in pipeline engine, we use parallel scan by default, but it broke the rule of data distribution - // so, if we do final phase or merge without exchange. - // we need turn of parallel scan to ensure to get correct result. - PlanNode leftMostNode = currentFragment.getPlanRoot(); - while (leftMostNode.getChildren().size() != 0 && !(leftMostNode instanceof ExchangeNode)) { - leftMostNode = leftMostNode.getChild(0); - } - // TODO: nereids forbid all parallel scan under aggregate temporary, because nereids could generate - // so complex aggregate plan than legacy planner, and should add forbid parallel scan hint when - // generate physical aggregate plan. - if (leftMostNode instanceof OlapScanNode - && currentFragment.getDataPartition().getType() != TPartitionType.RANDOM) { - currentFragment.setHasColocatePlanNode(true); - } - setPlanRoot(currentFragment, aggregationNode, aggregate); - if (aggregate.getStats() != null) { - aggregationNode.setCardinality((long) aggregate.getStats().getRowCount()); - } - updateLegacyPlanIdToPhysicalPlan(currentFragment.getPlanRoot(), aggregate); - return currentFragment; - } - - @Override - public PlanFragment visitPhysicalRepeat(PhysicalRepeat repeat, PlanTranslatorContext context) { - PlanFragment inputPlanFragment = repeat.child(0).accept(this, context); - - Set sortedVirtualSlots = repeat.getSortedVirtualSlots(); - TupleDescriptor virtualSlotsTuple = - generateTupleDesc(ImmutableList.copyOf(sortedVirtualSlots), null, context); - - ImmutableSet flattenGroupingSetExprs = ImmutableSet.copyOf( - ExpressionUtils.flatExpressions(repeat.getGroupingSets())); - - List aggregateFunctionUsedSlots = repeat.getOutputExpressions() - .stream() - .filter(output -> !(output instanceof VirtualSlotReference)) - .filter(output -> !flattenGroupingSetExprs.contains(output)) - .distinct() - .map(NamedExpression::toSlot) - .collect(ImmutableList.toImmutableList()); - - Set usedSlotInRepeat = ImmutableSet.builder() - .addAll(flattenGroupingSetExprs) - .addAll(aggregateFunctionUsedSlots) - .build(); - - List preRepeatExprs = usedSlotInRepeat.stream() - .map(expr -> ExpressionTranslator.translate(expr, context)) - .collect(ImmutableList.toImmutableList()); - - List outputSlots = repeat.getOutputExpressions() - .stream() - .map(NamedExpression::toSlot) - .collect(ImmutableList.toImmutableList()); - - // NOTE: we should first translate preRepeatExprs, then generate output tuple, - // or else the preRepeatExprs can not find the bottom slotRef and throw - // exception: invalid slot id - TupleDescriptor outputTuple = generateTupleDesc(outputSlots, null, context); - - // cube and rollup already convert to grouping sets in LogicalPlanBuilder.withAggregate() - GroupingInfo groupingInfo = new GroupingInfo( - GroupingType.GROUPING_SETS, virtualSlotsTuple, outputTuple, preRepeatExprs); - - List> repeatSlotIdList = repeat.computeRepeatSlotIdList(getSlotIdList(outputTuple)); - Set allSlotId = repeatSlotIdList.stream() - .flatMap(Set::stream) - .collect(ImmutableSet.toImmutableSet()); + // TODO: should not translate conjunct here. need a new attr in FileScanNode to save push down conjuncts. + fileScan.getConjuncts().stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .forEach(scanNode::addConjunct); + TableName tableName = new TableName(null, "", ""); + TableRef ref = new TableRef(tableName, null, null); + BaseTableRef tableRef = new BaseTableRef(ref, table, tableName); + tupleDescriptor.setRef(tableRef); - RepeatNode repeatNode = new RepeatNode(context.nextPlanNodeId(), - inputPlanFragment.getPlanRoot(), groupingInfo, repeatSlotIdList, - allSlotId, repeat.computeVirtualSlotValues(sortedVirtualSlots)); - repeatNode.setNumInstances(inputPlanFragment.getPlanRoot().getNumInstances()); - addPlanRoot(inputPlanFragment, repeatNode, repeat); - inputPlanFragment.updateDataPartition(DataPartition.RANDOM); - updateLegacyPlanIdToPhysicalPlan(inputPlanFragment.getPlanRoot(), repeat); - return inputPlanFragment; + Utils.execWithUncheckedException(scanNode::init); + context.addScanNode(scanNode); + ScanNode finalScanNode = scanNode; + context.getRuntimeTranslator().ifPresent( + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(fileScan.getId()).forEach( + expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, finalScanNode, context) + ) + ); + Utils.execWithUncheckedException(scanNode::finalizeForNereids); + // Create PlanFragment + DataPartition dataPartition = DataPartition.RANDOM; + PlanFragment planFragment = createPlanFragment(scanNode, dataPartition, fileScan); + context.addPlanFragment(planFragment); + updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), fileScan); + return planFragment; } @Override public PlanFragment visitPhysicalEmptyRelation(PhysicalEmptyRelation emptyRelation, PlanTranslatorContext context) { List output = emptyRelation.getOutput(); TupleDescriptor tupleDescriptor = generateTupleDesc(output, null, context); - for (int i = 0; i < output.size(); i++) { - Slot slot = output.get(i); + for (Slot slot : output) { SlotRef slotRef = context.findSlotRef(slot.getExprId()); slotRef.setLabel(slot.getName()); } @@ -547,89 +399,72 @@ public PlanFragment visitPhysicalEmptyRelation(PhysicalEmptyRelation emptyRelati } @Override - public PlanFragment visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, - PlanTranslatorContext context) { - if (oneRowRelation.notBuildUnionNode()) { - return null; - } - - List slots = oneRowRelation.getLogicalProperties().getOutput(); - TupleDescriptor oneRowTuple = generateTupleDesc(slots, null, context); - - List legacyExprs = oneRowRelation.getProjects() - .stream() - .map(expr -> ExpressionTranslator.translate(expr, context)) - .collect(Collectors.toList()); - - for (int i = 0; i < legacyExprs.size(); i++) { - SlotDescriptor slotDescriptor = oneRowTuple.getSlots().get(i); - Expr expr = legacyExprs.get(i); - slotDescriptor.setSourceExpr(expr); - slotDescriptor.setIsNullable(expr.isNullable()); - } - - UnionNode unionNode = new UnionNode(context.nextPlanNodeId(), oneRowTuple.getId()); - unionNode.setCardinality(1L); - unionNode.addConstExprList(legacyExprs); - unionNode.finalizeForNereids(oneRowTuple.getSlots(), new ArrayList<>()); - - PlanFragment planFragment = createPlanFragment(unionNode, DataPartition.UNPARTITIONED, oneRowRelation); + public PlanFragment visitPhysicalEsScan(PhysicalEsScan esScan, PlanTranslatorContext context) { + List slots = esScan.getOutput(); + ExternalTable table = esScan.getTable(); + TupleDescriptor tupleDescriptor = generateTupleDesc(slots, table, context); + EsScanNode esScanNode = new EsScanNode(context.nextPlanNodeId(), tupleDescriptor, true); + Utils.execWithUncheckedException(esScanNode::init); + context.addScanNode(esScanNode); + context.getRuntimeTranslator().ifPresent( + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(esScan.getId()).forEach( + expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, esScanNode, context) + ) + ); + Utils.execWithUncheckedException(esScanNode::finalizeForNereids); + DataPartition dataPartition = DataPartition.RANDOM; + PlanFragment planFragment = new PlanFragment(context.nextFragmentId(), esScanNode, dataPartition); context.addPlanFragment(planFragment); - updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), oneRowRelation); + updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), esScan); return planFragment; } @Override - public PlanFragment visitPhysicalStorageLayerAggregate( - PhysicalStorageLayerAggregate storageLayerAggregate, PlanTranslatorContext context) { - Preconditions.checkState(storageLayerAggregate.getRelation() instanceof PhysicalOlapScan, - "PhysicalStorageLayerAggregate only support PhysicalOlapScan: " - + storageLayerAggregate.getRelation().getClass().getName()); - PlanFragment planFragment = storageLayerAggregate.getRelation().accept(this, context); - - OlapScanNode olapScanNode = (OlapScanNode) planFragment.getPlanRoot(); - TPushAggOp pushAggOp; - switch (storageLayerAggregate.getAggOp()) { - case COUNT: - pushAggOp = TPushAggOp.COUNT; - break; - case MIN_MAX: - pushAggOp = TPushAggOp.MINMAX; - break; - case MIX: - pushAggOp = TPushAggOp.MIX; - break; - default: - throw new AnalysisException("Unsupported storage layer aggregate: " - + storageLayerAggregate.getAggOp()); - } - olapScanNode.setPushDownAggNoGrouping(pushAggOp); - updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), storageLayerAggregate); + public PlanFragment visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, PlanTranslatorContext context) { + List slots = jdbcScan.getOutput(); + TableIf table = jdbcScan.getTable(); + TupleDescriptor tupleDescriptor = generateTupleDesc(slots, table, context); + JdbcScanNode jdbcScanNode = new JdbcScanNode(context.nextPlanNodeId(), tupleDescriptor, + table instanceof JdbcExternalTable); + Utils.execWithUncheckedException(jdbcScanNode::init); + context.addScanNode(jdbcScanNode); + context.getRuntimeTranslator().ifPresent( + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(jdbcScan.getId()).forEach( + expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, jdbcScanNode, context) + ) + ); + Utils.execWithUncheckedException(jdbcScanNode::finalizeForNereids); + DataPartition dataPartition = DataPartition.RANDOM; + PlanFragment planFragment = new PlanFragment(context.nextFragmentId(), jdbcScanNode, dataPartition); + context.addPlanFragment(planFragment); + updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), jdbcScan); return planFragment; } @Override public PlanFragment visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanTranslatorContext context) { - // Create OlapScanNode - List slotList = olapScan.getOutput(); - Set deferredMaterializedExprIds = Collections.emptySet(); - if (olapScan.getMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS).isPresent()) { - deferredMaterializedExprIds = (Set) (olapScan - .getMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS).get()); - } - OlapTable olapTable = olapScan.getTable(); - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, olapTable, deferredMaterializedExprIds, context); + // deferred materialized slots used for topn opt. + Set deferredMaterializedExprIds = olapScan + .getMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS) + .map(s -> (Set) s) + .orElse(Collections.emptySet()); + List slots = olapScan.getOutput(); + OlapTable olapTable = olapScan.getTable(); + // generate real output tuple + TupleDescriptor tupleDescriptor = generateTupleDesc(slots, olapTable, deferredMaterializedExprIds, context); + // generate base index tuple because this fragment partitioned expr relay on slots of based index if (olapScan.getSelectedIndexId() != olapScan.getTable().getBaseIndexId()) { generateTupleDesc(olapScan.getBaseOutputs(), olapTable, deferredMaterializedExprIds, context); } + // TODO: remove this, we should add this column in Nereids if (olapScan.getMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS).isPresent()) { injectRowIdColumnSlot(tupleDescriptor); } - tupleDescriptor.setTable(olapTable); OlapScanNode olapScanNode = new OlapScanNode(context.nextPlanNodeId(), tupleDescriptor, "OlapScanNode"); + // TODO: move all node set cardinality into one place if (olapScan.getStats() != null) { olapScanNode.setCardinality((long) olapScan.getStats().getRowCount()); } @@ -641,6 +476,7 @@ public PlanFragment visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanTransla olapScanNode.setSelectedPartitionIds(olapScan.getSelectedPartitionIds()); olapScanNode.setSampleTabletIds(olapScan.getSelectedTabletIds()); + // TODO: remove this switch? switch (olapScan.getTable().getKeysType()) { case AGG_KEYS: case UNIQUE_KEYS: @@ -652,16 +488,21 @@ public PlanFragment visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanTransla throw new RuntimeException("Not supported key type: " + olapScan.getTable().getKeysType()); } + // create scan range Utils.execWithUncheckedException(olapScanNode::init); + // TODO: process collect scan node in one place context.addScanNode(olapScanNode); - // translate runtime filter + // TODO: process translate runtime filter in one place + // use real plan node to present rf apply and rf generator context.getRuntimeTranslator().ifPresent( runtimeFilterTranslator -> runtimeFilterTranslator.getTargetOnScanNode(olapScan.getId()).forEach( expr -> runtimeFilterTranslator.translateRuntimeFilterTarget(expr, olapScanNode, context) ) ); + // TODO: we need to remove all finalizeForNereids olapScanNode.finalizeForNereids(); // Create PlanFragment + // TODO: use a util function to convert distribution to DataPartition DataPartition dataPartition = DataPartition.RANDOM; if (olapScan.getDistributionSpec() instanceof DistributionSpecHash) { DistributionSpecHash distributionSpecHash = (DistributionSpecHash) olapScan.getDistributionSpec(); @@ -669,6 +510,7 @@ public PlanFragment visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanTransla .map(context::findSlotRef).collect(Collectors.toList()); dataPartition = new DataPartition(TPartitionType.HASH_PARTITIONED, partitionExprs); } + // TODO: maybe we could have a better way to create fragment PlanFragment planFragment = createPlanFragment(olapScanNode, dataPartition, olapScan); context.addPlanFragment(planFragment); updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), olapScan); @@ -676,78 +518,50 @@ public PlanFragment visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanTransla } @Override - public PlanFragment visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, PlanTranslatorContext context) { - Table table = schemaScan.getTable(); + public PlanFragment visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, + PlanTranslatorContext context) { + List slots = oneRowRelation.getLogicalProperties().getOutput(); + TupleDescriptor oneRowTuple = generateTupleDesc(slots, null, context); - List slotList = new ImmutableList.Builder() - .addAll(schemaScan.getOutput()) - .build(); - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, table, context); - tupleDescriptor.setTable(table); - SchemaScanNode scanNode = new SchemaScanNode(context.nextPlanNodeId(), tupleDescriptor); - context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(schemaScan.getId()).forEach( - expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) - ) - ); - scanNode.finalizeForNereids(); - context.getScanNodes().add(scanNode); - PlanFragment planFragment = createPlanFragment(scanNode, DataPartition.RANDOM, schemaScan); + List legacyExprs = oneRowRelation.getProjects() + .stream() + .map(expr -> ExpressionTranslator.translate(expr, context)) + .collect(Collectors.toList()); + + for (int i = 0; i < legacyExprs.size(); i++) { + SlotDescriptor slotDescriptor = oneRowTuple.getSlots().get(i); + Expr expr = legacyExprs.get(i); + slotDescriptor.setSourceExpr(expr); + slotDescriptor.setIsNullable(expr.isNullable()); + } + + UnionNode unionNode = new UnionNode(context.nextPlanNodeId(), oneRowTuple.getId()); + unionNode.setCardinality(1L); + unionNode.addConstExprList(legacyExprs); + unionNode.finalizeForNereids(oneRowTuple.getSlots(), new ArrayList<>()); + + PlanFragment planFragment = createPlanFragment(unionNode, DataPartition.UNPARTITIONED, oneRowRelation); context.addPlanFragment(planFragment); + updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), oneRowRelation); return planFragment; } @Override - public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTranslatorContext context) { - List slotList = fileScan.getOutput(); - ExternalTable table = fileScan.getTable(); - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, table, context); - tupleDescriptor.setTable(table); - - // TODO(cmy): determine the needCheckColumnPriv param - ScanNode scanNode = null; - if (table instanceof HMSExternalTable) { - switch (((HMSExternalTable) table).getDlaType()) { - case HUDI: - scanNode = new HudiScanNode(context.nextPlanNodeId(), tupleDescriptor, false); - break; - case ICEBERG: - scanNode = new IcebergScanNode(context.nextPlanNodeId(), tupleDescriptor, false); - break; - case HIVE: - scanNode = new HiveScanNode(context.nextPlanNodeId(), tupleDescriptor, false); - break; - default: - break; - } - } else if (table instanceof IcebergExternalTable) { - scanNode = new IcebergScanNode(context.nextPlanNodeId(), tupleDescriptor, false); - } else if (table instanceof PaimonExternalTable) { - scanNode = new PaimonScanNode(context.nextPlanNodeId(), tupleDescriptor, false); - } - Preconditions.checkNotNull(scanNode); - fileScan.getConjuncts().stream() - .map(e -> ExpressionTranslator.translate(e, context)) - .forEach(scanNode::addConjunct); - TableName tableName = new TableName(null, "", ""); - TableRef ref = new TableRef(tableName, null, null); - BaseTableRef tableRef = new BaseTableRef(ref, table, tableName); - tupleDescriptor.setRef(tableRef); - - Utils.execWithUncheckedException(scanNode::init); - context.addScanNode(scanNode); - ScanNode finalScanNode = scanNode; + public PlanFragment visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, PlanTranslatorContext context) { + Table table = schemaScan.getTable(); + List slots = ImmutableList.copyOf(schemaScan.getOutput()); + TupleDescriptor tupleDescriptor = generateTupleDesc(slots, table, context); + SchemaScanNode scanNode = new SchemaScanNode(context.nextPlanNodeId(), tupleDescriptor); context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(fileScan.getId()).forEach( - expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, finalScanNode, context) - ) + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(schemaScan.getId()).forEach( + expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) + ) ); - Utils.execWithUncheckedException(scanNode::finalizeForNereids); - // Create PlanFragment - DataPartition dataPartition = DataPartition.RANDOM; - PlanFragment planFragment = createPlanFragment(scanNode, dataPartition, fileScan); + scanNode.finalizeForNereids(); + context.addScanNode(scanNode); + PlanFragment planFragment = createPlanFragment(scanNode, DataPartition.RANDOM, schemaScan); context.addPlanFragment(planFragment); - updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), fileScan); + updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), schemaScan); return planFragment; } @@ -761,12 +575,13 @@ public PlanFragment visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, Pl Utils.execWithUncheckedException(scanNode::init); context.getRuntimeTranslator().ifPresent( runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(tvfRelation.getId()).forEach( - expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) - ) + expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) + ) ); Utils.execWithUncheckedException(scanNode::finalizeForNereids); context.addScanNode(scanNode); + // TODO: it is weird update label in this way // set label for explain for (Slot slot : slots) { String tableColumnName = "_table_valued_function_" + tvfRelation.getFunction().getName() @@ -776,433 +591,312 @@ public PlanFragment visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, Pl PlanFragment planFragment = createPlanFragment(scanNode, DataPartition.RANDOM, tvfRelation); context.addPlanFragment(planFragment); + updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), tvfRelation); return planFragment; } - @Override - public PlanFragment visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, PlanTranslatorContext context) { - List slotList = jdbcScan.getOutput(); - TableIf table = jdbcScan.getTable(); - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, table, context); - tupleDescriptor.setTable(table); - JdbcScanNode jdbcScanNode = new JdbcScanNode(context.nextPlanNodeId(), tupleDescriptor, - table instanceof JdbcExternalTable); - Utils.execWithUncheckedException(jdbcScanNode::init); - context.addScanNode(jdbcScanNode); - context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(jdbcScan.getId()).forEach( - expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, jdbcScanNode, context) - ) - ); - Utils.execWithUncheckedException(jdbcScanNode::finalizeForNereids); - DataPartition dataPartition = DataPartition.RANDOM; - PlanFragment planFragment = new PlanFragment(context.nextFragmentId(), jdbcScanNode, dataPartition); - context.addPlanFragment(planFragment); - updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), jdbcScan); - return planFragment; - } - @Override - public PlanFragment visitPhysicalEsScan(PhysicalEsScan esScan, PlanTranslatorContext context) { - List slotList = esScan.getOutput(); - ExternalTable table = esScan.getTable(); - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, table, context); - tupleDescriptor.setTable(table); - EsScanNode esScanNode = new EsScanNode(context.nextPlanNodeId(), tupleDescriptor, "EsScanNode", true); - Utils.execWithUncheckedException(esScanNode::init); - context.addScanNode(esScanNode); - context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(esScan.getId()).forEach( - expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, esScanNode, context) - ) - ); - Utils.execWithUncheckedException(esScanNode::finalizeForNereids); - DataPartition dataPartition = DataPartition.RANDOM; - PlanFragment planFragment = new PlanFragment(context.nextFragmentId(), esScanNode, dataPartition); - context.addPlanFragment(planFragment); - updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), esScan); - return planFragment; - } + /* ******************************************************************************************** + * other Node, in lexicographical order, ignore algorithm name. for example, HashAggregate -> Aggregate + * ******************************************************************************************** */ - /*- - * Physical sort: - * 1. Build sortInfo - * There are two types of slotRef: - * one is generated by the previous node, collectively called old. - * the other is newly generated by the sort node, collectively called new. - * Filling of sortInfo related data structures, - * a. ordering use newSlotRef. - * b. sortTupleSlotExprs use oldSlotRef. - * 2. Create sortNode - * 3. Create mergeFragment - * TODO: When the slotRef of sort is currently generated, - * it will be based on the expression in select and orderBy expression in to ensure the uniqueness of slotRef. - * But eg: - * select a+1 from table order by a+1; - * the expressions of the two are inconsistent. - * The former will perform an additional Alias. - * Currently we cannot test whether this will have any effect. - * After a+1 can be parsed , reprocessing. + /** + * Translate Agg. */ @Override - public PlanFragment visitPhysicalQuickSort(PhysicalQuickSort sort, + public PlanFragment visitPhysicalHashAggregate( + PhysicalHashAggregate aggregate, PlanTranslatorContext context) { - PlanFragment inputFragment = sort.child(0).accept(this, context); - PlanFragment currentFragment = inputFragment; - - //1. generate new fragment for sort when the child is exchangeNode - if (inputFragment.getPlanRoot() instanceof ExchangeNode - && sort.child(0) instanceof PhysicalDistribute) { - DataPartition outputPartition = DataPartition.UNPARTITIONED; - if (sort.getSortPhase().isLocal()) { - // The window function like over (partition by xx order by) can generate plan - // shuffle -> localSort -> windowFunction - // In this case, the child's partition need to keep - outputPartition = hashSpecToDataPartition((PhysicalDistribute) sort.child(0), context); - } - ExchangeNode exchangeNode = (ExchangeNode) inputFragment.getPlanRoot(); - inputFragment.setOutputPartition(outputPartition); - inputFragment.setPlanRoot(exchangeNode.getChild(0)); - currentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, outputPartition); - inputFragment.setDestination(exchangeNode); - context.addPlanFragment(currentFragment); - } - - // 2. According to the type of sort, generate physical plan - if (!sort.getSortPhase().isMerge()) { - // For localSort or Gather->Sort, we just need to add sortNode - SortNode sortNode = translateSortNode(sort, inputFragment.getPlanRoot(), context); - addPlanRoot(currentFragment, sortNode, sort); - } else { - // For mergeSort, we need to push sortInfo to exchangeNode - if (!(currentFragment.getPlanRoot() instanceof ExchangeNode)) { - // if there is no exchange node for mergeSort - // e.g., localSort -> mergeSort - // It means the local has satisfied the Gather property. We can just ignore mergeSort - return currentFragment; - } - Preconditions.checkArgument(inputFragment.getPlanRoot() instanceof SortNode); - SortNode sortNode = (SortNode) inputFragment.getPlanRoot(); - ((ExchangeNode) currentFragment.getPlanRoot()).setMergeInfo(sortNode.getSortInfo()); - } - return currentFragment; - } - @Override - public PlanFragment visitPhysicalWindow(PhysicalWindow physicalWindow, - PlanTranslatorContext context) { - PlanFragment inputPlanFragment = physicalWindow.child(0).accept(this, context); + PlanFragment inputPlanFragment = aggregate.child(0).accept(this, context); - // 1. translate to old optimizer variable - // variable in Nereids - WindowFrameGroup windowFrameGroup = physicalWindow.getWindowFrameGroup(); - List partitionKeyList = Lists.newArrayList(windowFrameGroup.getPartitionKeys()); - List orderKeyList = windowFrameGroup.getOrderKeys(); - List windowFunctionList = windowFrameGroup.getGroups(); - WindowFrame windowFrame = windowFrameGroup.getWindowFrame(); + List groupByExpressions = aggregate.getGroupByExpressions(); + List outputExpressions = aggregate.getOutputExpressions(); - // partition by clause - List partitionExprs = partitionKeyList.stream() + // 1. generate slot reference for each group expression + List groupSlots = collectGroupBySlots(groupByExpressions, outputExpressions); + ArrayList execGroupingExpressions = groupByExpressions.stream() .map(e -> ExpressionTranslator.translate(e, context)) + .collect(Collectors.toCollection(ArrayList::new)); + // 2. collect agg expressions and generate agg function to slot reference map + List aggFunctionOutput = Lists.newArrayList(); + List aggregateExpressionList = outputExpressions.stream() + .filter(o -> o.anyMatch(AggregateExpression.class::isInstance)) + .peek(o -> aggFunctionOutput.add(o.toSlot())) + .map(o -> o.>collect(AggregateExpression.class::isInstance)) + .flatMap(Set::stream) .collect(Collectors.toList()); + ArrayList execAggregateFunctions = aggregateExpressionList.stream() + .map(aggregateFunction -> (FunctionCallExpr) ExpressionTranslator.translate(aggregateFunction, context)) + .collect(Collectors.toCollection(ArrayList::new)); - // order by clause - List orderByElements = orderKeyList.stream() - .map(orderKey -> new OrderByElement( - ExpressionTranslator.translate(orderKey.child(), context), - orderKey.isAsc(), orderKey.isNullFirst())) - .collect(Collectors.toList()); + // 3. generate output tuple + List slotList = Lists.newArrayList(); + TupleDescriptor outputTupleDesc; + slotList.addAll(groupSlots); + slotList.addAll(aggFunctionOutput); + outputTupleDesc = generateTupleDesc(slotList, null, context); - // function calls - List analyticFnCalls = windowFunctionList.stream() - .map(e -> { - Expression function = e.child(0).child(0); - if (function instanceof AggregateFunction) { - AggregateParam param = AggregateParam.localResult(); - function = new AggregateExpression((AggregateFunction) function, param); - } - return ExpressionTranslator.translate(function, context); - }) - .map(FunctionCallExpr.class::cast) - .map(fnCall -> { - fnCall.setIsAnalyticFnCall(true); - ((org.apache.doris.catalog.AggregateFunction) fnCall.getFn()).setIsAnalyticFn(true); - return fnCall; - }) - .collect(Collectors.toList()); + List aggFunOutputIds = ImmutableList.of(); + if (!aggFunctionOutput.isEmpty()) { + aggFunOutputIds = outputTupleDesc + .getSlots() + .subList(groupSlots.size(), outputTupleDesc.getSlots().size()) + .stream() + .map(slot -> slot.getId().asInt()) + .collect(ImmutableList.toImmutableList()); + } + boolean isPartial = aggregate.getAggregateParam().aggMode.productAggregateBuffer; + AggregateInfo aggInfo = AggregateInfo.create(execGroupingExpressions, execAggregateFunctions, + aggFunOutputIds, isPartial, outputTupleDesc, outputTupleDesc, aggregate.getAggPhase().toExec()); + AggregationNode aggregationNode = new AggregationNode(context.nextPlanNodeId(), + inputPlanFragment.getPlanRoot(), aggInfo); + if (!aggregate.getAggMode().isFinalPhase) { + aggregationNode.unsetNeedsFinalize(); + } - // analytic window - AnalyticWindow analyticWindow = physicalWindow.translateWindowFrame(windowFrame, context); + switch (aggregate.getAggPhase()) { + case LOCAL: + // we should set is useStreamingAgg when has exchange, + // so the `aggregationNode.setUseStreamingPreagg()` in the visitPhysicalDistribute + break; + case DISTINCT_LOCAL: + aggregationNode.setIntermediateTuple(); + break; + case GLOBAL: + case DISTINCT_GLOBAL: + break; + default: + throw new RuntimeException("Unsupported agg phase: " + aggregate.getAggPhase()); + } + // TODO: use to set useStreamingAgg, we should remove it by set it in Nereids + PhysicalHashAggregate firstAggregateInFragment = context.getFirstAggregateInFragment(inputPlanFragment); + if (firstAggregateInFragment == null) { + context.setFirstAggregateInFragment(inputPlanFragment, aggregate); + } - // 2. get bufferedTupleDesc from SortNode and compute isNullableMatched - Map bufferedSlotRefForWindow = getBufferedSlotRefForWindow(windowFrameGroup, context); - TupleDescriptor bufferedTupleDesc = context.getBufferedTupleForWindow(); + // in pipeline engine, we use parallel scan by default, but it broke the rule of data distribution + // so, if we do final phase or merge without exchange. + // we need turn of parallel scan to ensure to get correct result. + PlanNode leftMostNode = inputPlanFragment.getPlanRoot(); + while (leftMostNode.getChildren().size() != 0 && !(leftMostNode instanceof ExchangeNode)) { + leftMostNode = leftMostNode.getChild(0); + } + // TODO: nereids forbid all parallel scan under aggregate temporary, because nereids could generate + // so complex aggregate plan than legacy planner, and should add forbid parallel scan hint when + // generate physical aggregate plan. + if (leftMostNode instanceof OlapScanNode + && inputPlanFragment.getDataPartition().getType() != TPartitionType.RANDOM) { + inputPlanFragment.setHasColocatePlanNode(true); + } + setPlanRoot(inputPlanFragment, aggregationNode, aggregate); + if (aggregate.getStats() != null) { + aggregationNode.setCardinality((long) aggregate.getStats().getRowCount()); + } + updateLegacyPlanIdToPhysicalPlan(inputPlanFragment.getPlanRoot(), aggregate); + return inputPlanFragment; + } - // generate predicates to check if the exprs of partitionKeys and orderKeys have matched isNullable between - // sortNode and analyticNode - Expr partitionExprsIsNullableMatched = partitionExprs.isEmpty() ? null : windowExprsHaveMatchedNullable( - partitionKeyList, partitionExprs, bufferedSlotRefForWindow); + @Override + public PlanFragment visitPhysicalStorageLayerAggregate( + PhysicalStorageLayerAggregate storageLayerAggregate, PlanTranslatorContext context) { + Preconditions.checkState(storageLayerAggregate.getRelation() instanceof PhysicalOlapScan, + "PhysicalStorageLayerAggregate only support PhysicalOlapScan: " + + storageLayerAggregate.getRelation().getClass().getName()); + PlanFragment planFragment = storageLayerAggregate.getRelation().accept(this, context); - Expr orderElementsIsNullableMatched = orderByElements.isEmpty() ? null : windowExprsHaveMatchedNullable( - orderKeyList.stream().map(order -> order.child()).collect(Collectors.toList()), - orderByElements.stream().map(order -> order.getExpr()).collect(Collectors.toList()), - bufferedSlotRefForWindow); + OlapScanNode olapScanNode = (OlapScanNode) planFragment.getPlanRoot(); + TPushAggOp pushAggOp; + switch (storageLayerAggregate.getAggOp()) { + case COUNT: + pushAggOp = TPushAggOp.COUNT; + break; + case MIN_MAX: + pushAggOp = TPushAggOp.MINMAX; + break; + case MIX: + pushAggOp = TPushAggOp.MIX; + break; + default: + throw new AnalysisException("Unsupported storage layer aggregate: " + + storageLayerAggregate.getAggOp()); + } + olapScanNode.setPushDownAggNoGrouping(pushAggOp); + updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), storageLayerAggregate); + return planFragment; + } - // 3. generate tupleDesc - List windowSlotList = windowFunctionList.stream() - .map(NamedExpression::toSlot) - .collect(Collectors.toList()); - TupleDescriptor outputTupleDesc = generateTupleDesc(windowSlotList, null, context); - TupleDescriptor intermediateTupleDesc = outputTupleDesc; + @Override + public PlanFragment visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, + PlanTranslatorContext context) { + PlanFragment currentFragment = assertNumRows.child().accept(this, context); + // create assertNode + AssertNumRowsNode assertNumRowsNode = new AssertNumRowsNode(context.nextPlanNodeId(), + currentFragment.getPlanRoot(), + ExpressionTranslator.translateAssert(assertNumRows.getAssertNumRowsElement())); + addPlanRoot(currentFragment, assertNumRowsNode, assertNumRows); + return currentFragment; + } - // 4. generate AnalyticEvalNode - AnalyticEvalNode analyticEvalNode = new AnalyticEvalNode( - context.nextPlanNodeId(), - inputPlanFragment.getPlanRoot(), - analyticFnCalls, - partitionExprs, - orderByElements, - analyticWindow, - intermediateTupleDesc, - outputTupleDesc, - partitionExprsIsNullableMatched, - orderElementsIsNullableMatched, - bufferedTupleDesc - ); + /** + * NOTICE: Must translate left, which it's the producer of consumer. + */ + @Override + public PlanFragment visitPhysicalCTEAnchor(PhysicalCTEAnchor cteAnchor, + PlanTranslatorContext context) { + cteAnchor.child(0).accept(this, context); + return cteAnchor.child(1).accept(this, context); + } - if (partitionExprs.isEmpty() && orderByElements.isEmpty()) { - if (inputPlanFragment.isPartitioned()) { - PlanFragment parentFragment = new PlanFragment(context.nextFragmentId(), analyticEvalNode, - DataPartition.UNPARTITIONED); - context.addPlanFragment(parentFragment); - connectChildFragment(analyticEvalNode, 0, parentFragment, inputPlanFragment, context); - inputPlanFragment = parentFragment; + @Override + public PlanFragment visitPhysicalCTEConsumer(PhysicalCTEConsumer cteConsumer, + PlanTranslatorContext context) { + CTEId cteId = cteConsumer.getCteId(); + + MultiCastPlanFragment multiCastFragment = (MultiCastPlanFragment) context.getCteProduceFragments().get(cteId); + Preconditions.checkState(multiCastFragment.getSink() instanceof MultiCastDataSink, + "invalid multiCastFragment"); + + MultiCastDataSink multiCastDataSink = (MultiCastDataSink) multiCastFragment.getSink(); + Preconditions.checkState(multiCastDataSink != null, "invalid multiCastDataSink"); + + PhysicalCTEProducer cteProducer = context.getCteProduceMap().get(cteId); + Preconditions.checkState(cteProducer != null, "invalid cteProducer"); + + ExchangeNode exchangeNode = new ExchangeNode(context.nextPlanNodeId(), multiCastFragment.getPlanRoot()); + + DataStreamSink streamSink = new DataStreamSink(exchangeNode.getId()); + streamSink.setPartition(DataPartition.RANDOM); + streamSink.setFragment(multiCastFragment); + + multiCastDataSink.getDataStreamSinks().add(streamSink); + multiCastDataSink.getDestinations().add(Lists.newArrayList()); + + exchangeNode.setNumInstances(multiCastFragment.getPlanRoot().getNumInstances()); + + PlanFragment consumeFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, + multiCastFragment.getDataPartition()); + + Map projectMap = Maps.newHashMap(); + projectMap.putAll(cteConsumer.getProducerToConsumerSlotMap()); + + List execList = new ArrayList<>(); + PlanNode inputPlanNode = consumeFragment.getPlanRoot(); + List cteProjects = cteProducer.getProjects(); + for (Slot slot : cteProjects) { + if (projectMap.containsKey(slot)) { + execList.add(projectMap.get(slot)); } else { - inputPlanFragment.addPlanRoot(analyticEvalNode); + throw new RuntimeException("could not find slot in cte producer consumer projectMap"); } - } else { - analyticEvalNode.setNumInstances(inputPlanFragment.getPlanRoot().getNumInstances()); - inputPlanFragment.addPlanRoot(analyticEvalNode); } - return inputPlanFragment; - } - @Override - public PlanFragment visitPhysicalPartitionTopN(PhysicalPartitionTopN partitionTopN, - PlanTranslatorContext context) { - PlanFragment inputFragment = partitionTopN.child(0).accept(this, context); + List slotList = execList + .stream() + .map(NamedExpression::toSlot) + .collect(Collectors.toList()); - Preconditions.checkArgument(!(partitionTopN.child(0) instanceof ExchangeNode)); - PartitionSortNode partitionSortNode = translatePartitionSortNode(partitionTopN, - inputFragment.getPlanRoot(), context); - addPlanRoot(inputFragment, partitionSortNode, partitionTopN); + TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, null, context); - return inputFragment; - } + // update tuple list and tblTupleList + inputPlanNode.getTupleIds().clear(); + inputPlanNode.getTupleIds().add(tupleDescriptor.getId()); + inputPlanNode.getTblRefIds().clear(); + inputPlanNode.getTblRefIds().add(tupleDescriptor.getId()); + inputPlanNode.getNullableTupleIds().clear(); + inputPlanNode.getNullableTupleIds().add(tupleDescriptor.getId()); - private PartitionSortNode translatePartitionSortNode(PhysicalPartitionTopN partitionTopN, - PlanNode childNode, PlanTranslatorContext context) { - // Generate the SortInfo, similar to 'translateSortNode'. - List oldOrderingExprList = Lists.newArrayList(); - List ascOrderList = Lists.newArrayList(); - List nullsFirstParamList = Lists.newArrayList(); - List orderKeyList = partitionTopN.getOrderKeys(); - // 1. Get previous slotRef - orderKeyList.forEach(k -> { - oldOrderingExprList.add(ExpressionTranslator.translate(k.getExpr(), context)); - ascOrderList.add(k.isAsc()); - nullsFirstParamList.add(k.isNullFirst()); - }); - List sortTupleOutputList = new ArrayList<>(); - List outputList = partitionTopN.getOutput(); - outputList.forEach(k -> { - sortTupleOutputList.add(ExpressionTranslator.translate(k, context)); - }); - List partitionExprs = partitionTopN.getPartitionKeys().stream() + List execExprList = execList + .stream() .map(e -> ExpressionTranslator.translate(e, context)) .collect(Collectors.toList()); - // 2. Generate new Tuple and get current slotRef for newOrderingExprList - List newOrderingExprList = Lists.newArrayList(); - TupleDescriptor tupleDesc = generateTupleDesc(outputList, orderKeyList, newOrderingExprList, context, null); - // 3. fill in SortInfo members - SortInfo sortInfo = new SortInfo(newOrderingExprList, ascOrderList, nullsFirstParamList, tupleDesc); - PartitionSortNode partitionSortNode = new PartitionSortNode(context.nextPlanNodeId(), childNode, - partitionTopN.getFunction(), partitionExprs, sortInfo, partitionTopN.hasGlobalLimit(), - partitionTopN.getPartitionLimit(), sortTupleOutputList, oldOrderingExprList); + inputPlanNode.setProjectList(execExprList); + inputPlanNode.setOutputTupleDesc(tupleDescriptor); - if (partitionTopN.getStats() != null) { - partitionSortNode.setCardinality((long) partitionTopN.getStats().getRowCount()); - } - updateLegacyPlanIdToPhysicalPlan(partitionSortNode, partitionTopN); - return partitionSortNode; + // update data partition + consumeFragment.setDataPartition(DataPartition.RANDOM); + + SelectNode projectNode = new SelectNode(context.nextPlanNodeId(), inputPlanNode); + consumeFragment.setPlanRoot(projectNode); + + multiCastFragment.getDestNodeList().add(exchangeNode); + consumeFragment.addChild(multiCastFragment); + context.getPlanFragments().add(consumeFragment); + + return consumeFragment; } @Override - public PlanFragment visitPhysicalTopN(PhysicalTopN topN, PlanTranslatorContext context) { - PlanFragment inputFragment = topN.child(0).accept(this, context); - PlanFragment currentFragment = inputFragment; - - //1. Generate new fragment for sort when the child is exchangeNode, If the child is - // mergingExchange, it means we have always generated a new fragment when processing mergeSort - if (inputFragment.getPlanRoot() instanceof ExchangeNode - && !((ExchangeNode) inputFragment.getPlanRoot()).isMergingExchange()) { - // Except LocalTopN->MergeTopN, we don't allow localTopN's child is Exchange Node - Preconditions.checkArgument(!topN.getSortPhase().isLocal(), - "local sort requires any property but child is" + inputFragment.getPlanRoot()); - DataPartition outputPartition = DataPartition.UNPARTITIONED; - ExchangeNode exchangeNode = (ExchangeNode) inputFragment.getPlanRoot(); - inputFragment.setOutputPartition(outputPartition); - inputFragment.setPlanRoot(exchangeNode.getChild(0)); - inputFragment.setDestination(exchangeNode); - currentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, DataPartition.UNPARTITIONED); - context.addPlanFragment(currentFragment); - } + public PlanFragment visitPhysicalCTEProducer(PhysicalCTEProducer cteProducer, + PlanTranslatorContext context) { + PlanFragment child = cteProducer.child().accept(this, context); + CTEId cteId = cteProducer.getCteId(); + context.getPlanFragments().remove(child); + MultiCastPlanFragment cteProduce = new MultiCastPlanFragment(child); + MultiCastDataSink multiCastDataSink = new MultiCastDataSink(); + cteProduce.setSink(multiCastDataSink); - // 2. According to the type of sort, generate physical plan - if (!topN.getSortPhase().isMerge()) { - // For localSort or Gather->Sort, we just need to add TopNNode - SortNode sortNode = translateSortNode(topN, inputFragment.getPlanRoot(), context); - sortNode.setOffset(topN.getOffset()); - sortNode.setLimit(topN.getLimit()); - if (topN.getMutableState(PhysicalTopN.TOPN_RUNTIME_FILTER).isPresent()) { - sortNode.setUseTopnOpt(true); - PlanNode child = sortNode.getChild(0); - Preconditions.checkArgument(child instanceof OlapScanNode, - "topN opt expect OlapScanNode, but we get " + child); - OlapScanNode scanNode = ((OlapScanNode) child); - scanNode.setUseTopnOpt(true); - } - // push sort to scan opt - if (sortNode.getChild(0) instanceof OlapScanNode) { - OlapScanNode scanNode = ((OlapScanNode) sortNode.getChild(0)); - if (checkPushSort(sortNode, scanNode.getOlapTable())) { - SortInfo sortInfo = sortNode.getSortInfo(); - scanNode.setSortInfo(sortInfo); - scanNode.getSortInfo().setSortTupleSlotExprs(sortNode.getResolvedTupleExprs()); - for (Expr expr : sortInfo.getOrderingExprs()) { - scanNode.getSortInfo().addMaterializedOrderingExpr(expr); - } - if (sortNode.getOffset() > 0) { - scanNode.setSortLimit(sortNode.getLimit() + sortNode.getOffset()); - } else { - scanNode.setSortLimit(sortNode.getLimit()); - } - } - } - addPlanRoot(currentFragment, sortNode, topN); - } else { - // For mergeSort, we need to push sortInfo to exchangeNode - if (!(currentFragment.getPlanRoot() instanceof ExchangeNode)) { - // if there is no exchange node for mergeSort - // e.g., mergeTopN -> localTopN - // It means the local has satisfied the Gather property. We can just ignore mergeSort - currentFragment.getPlanRoot().setOffset(topN.getOffset()); - currentFragment.getPlanRoot().setLimit(topN.getLimit()); - return currentFragment; - } - Preconditions.checkArgument(inputFragment.getPlanRoot() instanceof SortNode, - "mergeSort' child must be sortNode"); - SortNode sortNode = (SortNode) inputFragment.getPlanRoot(); - ExchangeNode exchangeNode = (ExchangeNode) currentFragment.getPlanRoot(); - exchangeNode.setMergeInfo(sortNode.getSortInfo()); - exchangeNode.setLimit(topN.getLimit()); - exchangeNode.setOffset(topN.getOffset()); - } - updateLegacyPlanIdToPhysicalPlan(currentFragment.getPlanRoot(), topN); - return currentFragment; - } + List outputs = cteProducer.getProjects().stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(Collectors.toList()); - /** - * topN opt: using storage data ordering to accelerate topn operation. - * refer pr: optimize topn query if order by columns is prefix of sort keys of table (#10694) - */ - public boolean checkPushSort(SortNode sortNode, OlapTable olapTable) { - // Ensure limit is less then threshold - if (sortNode.getLimit() <= 0 - || sortNode.getLimit() > ConnectContext.get().getSessionVariable().topnOptLimitThreshold) { - return false; - } + cteProduce.setOutputExprs(outputs); + context.getCteProduceFragments().put(cteId, cteProduce); + context.getCteProduceMap().put(cteId, cteProducer); + context.getPlanFragments().add(cteProduce); + return child; + } - // Ensure all isAscOrder is same, ande length != 0. - // Can't be zorder. - if (sortNode.getSortInfo().getIsAscOrder().stream().distinct().count() != 1 - || olapTable.isZOrderSort()) { - return false; + @Override + public PlanFragment visitPhysicalFilter(PhysicalFilter filter, PlanTranslatorContext context) { + if (filter.child(0) instanceof AbstractPhysicalJoin) { + AbstractPhysicalJoin join = (AbstractPhysicalJoin) filter.child(); + join.addFilterConjuncts(filter.getConjuncts()); } + PlanFragment inputFragment = filter.child(0).accept(this, context); - // Tablet's order by key only can be the front part of schema. - // Like: schema: a.b.c.d.e.f.g order by key: a.b.c (no a,b,d) - // Do **prefix match** to check if order by key can be pushed down. - // olap order by key: a.b.c.d - // sort key: (a) (a,b) (a,b,c) (a,b,c,d) is ok - // (a,c) (a,c,d), (a,c,b) (a,c,f) (a,b,c,d,e)is NOT ok - List sortExprs = sortNode.getSortInfo().getOrderingExprs(); - List nullsFirsts = sortNode.getSortInfo().getNullsFirst(); - List isAscOrders = sortNode.getSortInfo().getIsAscOrder(); - if (sortExprs.size() > olapTable.getDataSortInfo().getColNum()) { - return false; - } - for (int i = 0; i < sortExprs.size(); i++) { - // table key. - Column tableKey = olapTable.getFullSchema().get(i); - // sort slot. - Expr sortExpr = sortExprs.get(i); - if (sortExpr instanceof SlotRef) { - SlotRef slotRef = (SlotRef) sortExpr; - if (tableKey.equals(slotRef.getColumn())) { - // ORDER BY DESC NULLS FIRST can not be optimized to only read file tail, - // since NULLS is at file head but data is at tail - if (tableKey.isAllowNull() && nullsFirsts.get(i) && !isAscOrders.get(i)) { - return false; - } - } else { - return false; - } - } else { - return false; + PlanNode planNode = inputFragment.getPlanRoot(); + if (planNode instanceof ExchangeNode || planNode instanceof SortNode || planNode instanceof UnionNode) { + // the three nodes don't support conjuncts, need create a SelectNode to filter data + SelectNode selectNode = new SelectNode(context.nextPlanNodeId(), planNode); + addConjunctsToPlanNode(filter, selectNode, context); + addPlanRoot(inputFragment, selectNode, filter); + } else { + if (!(filter.child(0) instanceof AbstractPhysicalJoin)) { + addConjunctsToPlanNode(filter, planNode, context); + updateLegacyPlanIdToPhysicalPlan(inputFragment.getPlanRoot(), filter); } } - - return true; - } - - private SortNode translateSortNode(AbstractPhysicalSort sort, PlanNode childNode, - PlanTranslatorContext context) { - List oldOrderingExprList = Lists.newArrayList(); - List ascOrderList = Lists.newArrayList(); - List nullsFirstParamList = Lists.newArrayList(); - List orderKeyList = sort.getOrderKeys(); - // 1. Get previous slotRef - orderKeyList.forEach(k -> { - oldOrderingExprList.add(ExpressionTranslator.translate(k.getExpr(), context)); - ascOrderList.add(k.isAsc()); - nullsFirstParamList.add(k.isNullFirst()); - }); - List sortTupleOutputList = new ArrayList<>(); - List outputList = sort.getOutput(); - outputList.forEach(k -> sortTupleOutputList.add(ExpressionTranslator.translate(k, context))); - // 2. Generate new Tuple and get current slotRef for newOrderingExprList - List newOrderingExprList = Lists.newArrayList(); - TupleDescriptor tupleDesc = generateTupleDesc(outputList, orderKeyList, newOrderingExprList, context, null); - // 3. fill in SortInfo members - SortInfo sortInfo = new SortInfo(newOrderingExprList, ascOrderList, nullsFirstParamList, tupleDesc); - SortNode sortNode = new SortNode(context.nextPlanNodeId(), childNode, sortInfo, true); - sortNode.finalizeForNereids(tupleDesc, sortTupleOutputList, oldOrderingExprList); - if (sort.getMutableState(PhysicalTopN.TWO_PHASE_READ_OPT).isPresent()) { - sortNode.setUseTwoPhaseReadOpt(true); - sortNode.getSortInfo().setUseTwoPhaseRead(); - injectRowIdColumnSlot(sortNode.getSortInfo().getSortTupleDescriptor()); - TupleDescriptor childTuple = childNode.getOutputTupleDesc() != null - ? childNode.getOutputTupleDesc() : context.getTupleDesc(childNode.getTupleIds().get(0)); - SlotDescriptor childRowIdDesc = childTuple.getSlots().get(childTuple.getSlots().size() - 1); - sortNode.getResolvedTupleExprs().add(new SlotRef(childRowIdDesc)); + //in ut, filter.stats may be null + if (filter.getStats() != null) { + inputFragment.getPlanRoot().setCardinalityAfterFilter((long) filter.getStats().getRowCount()); } - if (sort.getStats() != null) { - sortNode.setCardinality((long) sort.getStats().getRowCount()); + return inputFragment; + } + + @Override + public PlanFragment visitPhysicalGenerate(PhysicalGenerate generate, + PlanTranslatorContext context) { + PlanFragment currentFragment = generate.child().accept(this, context); + ArrayList functionCalls = generate.getGenerators().stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(Collectors.toCollection(ArrayList::new)); + TupleDescriptor tupleDescriptor = generateTupleDesc(generate.getGeneratorOutput(), null, context); + List childOutputTupleIds = currentFragment.getPlanRoot().getOutputTupleIds(); + if (childOutputTupleIds == null || childOutputTupleIds.isEmpty()) { + childOutputTupleIds = currentFragment.getPlanRoot().getTupleIds(); } - updateLegacyPlanIdToPhysicalPlan(sortNode, sort); - return sortNode; + List outputSlotIds = Stream.concat(childOutputTupleIds.stream(), + Stream.of(tupleDescriptor.getId())) + .map(id -> context.getTupleDesc(id).getSlots()) + .flatMap(List::stream) + .map(SlotDescriptor::getId) + .collect(Collectors.toList()); + TableFunctionNode tableFunctionNode = new TableFunctionNode(context.nextPlanNodeId(), + currentFragment.getPlanRoot(), tupleDescriptor.getId(), functionCalls, outputSlotIds); + addPlanRoot(currentFragment, tableFunctionNode, generate); + return currentFragment; } /** @@ -1211,44 +905,44 @@ private SortNode translateSortNode(AbstractPhysicalSort sort, Pl * a. equal join conjuncts * b. other join conjuncts * c. other predicates (denoted by filter conjuncts in the rest of comments) - * + *

* 2. hash join contains 3 tuple descriptors * a. input tuple descriptors, corresponding to the left child output and right child output. * If its column is selected, it will be displayed in explain by `tuple ids`. * for example, select L.* from L join R on ..., because no column from R are selected, tuple ids only * contains output tuple of L. * equal join conjuncts is bound on input tuple descriptors. - * + *

* b.intermediate tuple. * This tuple describes schema of the output block after evaluating equal join conjuncts * and other join conjuncts. - * + *

* Other join conjuncts currently is bound on intermediate tuple. There are some historical reason, and it * should be bound on input tuple in the future. - * + *

* filter conjuncts will be evaluated on the intermediate tuple. That means the input block of filter is * described by intermediate tuple, and hence filter conjuncts should be bound on intermediate tuple. - * + *

* In order to be compatible with old version, intermediate tuple is not pruned. For example, intermediate * tuple contains all slots from both sides of children. After probing hash-table, BE does not need to * materialize all slots in intermediate tuple. The slots in HashJoinNode.hashOutputSlotIds will be * materialized by BE. If `hashOutputSlotIds` is empty, all slots will be materialized. - * + *

* In case of outer join, the slots in intermediate should be set nullable. * For example, * select L.*, R.* from L left outer join R on ... * All slots from R in intermediate tuple should be nullable. - * + *

* c. output tuple * This describes the schema of hash join output block. * 3. Intermediate tuple - * for BE performance reason, the slots in intermediate tuple depends on the join type and other join conjucts. + * for BE performance reason, the slots in intermediate tuple + * depends on the join type and other join conjuncts. * In general, intermediate tuple contains all slots of both children, except one case. * For left-semi/left-ant (right-semi/right-semi) join without other join conjuncts, intermediate tuple * only contains left (right) children output slots. * */ - // TODO: 1. support shuffle join / co-locate / bucket shuffle join later @Override public PlanFragment visitPhysicalHashJoin( PhysicalHashJoin hashJoin, @@ -1281,17 +975,22 @@ public PlanFragment visitPhysicalHashJoin( rightPlanRoot, JoinType.toJoinOperator(joinType), execEqConjuncts, Lists.newArrayList(), null, null, null, hashJoin.isMarkJoin()); - PlanFragment currentFragment; + PlanFragment currentFragment = connectJoinNode(hashJoinNode, leftFragment, rightFragment, context, hashJoin); if (JoinUtils.shouldColocateJoin(physicalHashJoin)) { - currentFragment = constructColocateJoin(hashJoinNode, leftFragment, rightFragment, context, hashJoin); - } else if (JoinUtils.shouldBucketShuffleJoin(physicalHashJoin)) { - currentFragment = constructBucketShuffleJoin( - physicalHashJoin, hashJoinNode, leftFragment, rightFragment, context); + // TODO: add reason + hashJoinNode.setColocate(true, ""); + leftFragment.setHasColocatePlanNode(true); } else if (JoinUtils.shouldBroadcastJoin(physicalHashJoin)) { - currentFragment = constructBroadcastJoin(hashJoinNode, leftFragment, rightFragment, context, hashJoin); + Preconditions.checkState(rightFragment.getPlanRoot() instanceof ExchangeNode, + "right child of broadcast join must be ExchangeNode but it is " + rightFragment.getPlanRoot()); + Preconditions.checkState(rightFragment.getChildren().size() == 1, + "right child of broadcast join must have 1 child, but meet " + rightFragment.getChildren().size()); + rightFragment.getChild(0).setRightChildOfBroadcastHashJoin(true); + hashJoinNode.setDistributionMode(DistributionMode.BROADCAST); + } else if (JoinUtils.shouldBucketShuffleJoin(physicalHashJoin)) { + hashJoinNode.setDistributionMode(DistributionMode.BUCKET_SHUFFLE); } else { - currentFragment = constructShuffleJoin( - physicalHashJoin, hashJoinNode, leftFragment, rightFragment, context); + hashJoinNode.setDistributionMode(DistributionMode.PARTITIONED); } // Nereids does not care about output order of join, // but BE need left child's output must be before right child's output. @@ -1500,17 +1199,14 @@ public PlanFragment visitPhysicalNestedLoopJoin( if (nestedLoopJoin.getStats() != null) { nestedLoopJoinNode.setCardinality((long) nestedLoopJoin.getStats().getRowCount()); } - boolean needNewRootFragment = nestedLoopJoin.child(0) instanceof PhysicalDistribute; - PlanFragment joinFragment; - if (needNewRootFragment) { - joinFragment = createPlanFragment(nestedLoopJoinNode, - DataPartition.UNPARTITIONED, nestedLoopJoin); - context.addPlanFragment(joinFragment); - connectChildFragment(nestedLoopJoinNode, 0, joinFragment, leftFragment, context); - } else { - joinFragment = leftFragment; - nestedLoopJoinNode.setChild(0, leftFragment.getPlanRoot()); - setPlanRoot(joinFragment, nestedLoopJoinNode, nestedLoopJoin); + nestedLoopJoinNode.setChild(0, leftFragment.getPlanRoot()); + nestedLoopJoinNode.setChild(1, rightFragment.getPlanRoot()); + setPlanRoot(leftFragment, nestedLoopJoinNode, nestedLoopJoin); + // TODO: what's this? do we really need to set this? + rightFragment.getPlanRoot().setCompactData(false); + context.removePlanFragment(rightFragment); + for (PlanFragment rightChild : rightFragment.getChildren()) { + leftFragment.addChild(rightChild); } // translate runtime filter context.getRuntimeTranslator().ifPresent(runtimeFilterTranslator -> { @@ -1612,9 +1308,6 @@ public PlanFragment visitPhysicalNestedLoopJoin( nestedLoopJoinNode.setvIntermediateTupleDescList(Lists.newArrayList(intermediateDescriptor)); - rightFragment.getPlanRoot().setCompactData(false); - - connectChildFragment(nestedLoopJoinNode, 1, joinFragment, rightFragment, context); List joinConjuncts = nestedLoopJoin.getOtherJoinConjuncts().stream() .filter(e -> !nestedLoopJoin.isBitmapRuntimeFilterCondition(e)) .map(e -> ExpressionTranslator.translate(e, context)).collect(Collectors.toList()); @@ -1646,28 +1339,42 @@ public PlanFragment visitPhysicalNestedLoopJoin( if (nestedLoopJoin.getStats() != null) { nestedLoopJoinNode.setCardinality((long) nestedLoopJoin.getStats().getRowCount()); } - updateLegacyPlanIdToPhysicalPlan(joinFragment.getPlanRoot(), nestedLoopJoin); - return joinFragment; + updateLegacyPlanIdToPhysicalPlan(leftFragment.getPlanRoot(), nestedLoopJoin); + return leftFragment; } else { throw new RuntimeException("Physical nested loop join could not execute with equal join condition."); } } + @Override + public PlanFragment visitPhysicalLimit(PhysicalLimit physicalLimit, PlanTranslatorContext context) { + PlanFragment inputFragment = physicalLimit.child(0).accept(this, context); + PlanNode child = inputFragment.getPlanRoot(); + child.setOffset(physicalLimit.getOffset()); + child.setLimit(physicalLimit.getLimit()); + updateLegacyPlanIdToPhysicalPlan(child, physicalLimit); + return inputFragment; + } + + @Override + public PlanFragment visitPhysicalPartitionTopN(PhysicalPartitionTopN partitionTopN, + PlanTranslatorContext context) { + PlanFragment inputFragment = partitionTopN.child(0).accept(this, context); + PartitionSortNode partitionSortNode = translatePartitionSortNode( + partitionTopN, inputFragment.getPlanRoot(), context); + addPlanRoot(inputFragment, partitionSortNode, partitionTopN); + return inputFragment; + } + // TODO: generate expression mapping when be project could do in ExecNode. @Override public PlanFragment visitPhysicalProject(PhysicalProject project, PlanTranslatorContext context) { - if (project.child(0) instanceof PhysicalHashJoin) { - ((PhysicalHashJoin) project.child(0)).setShouldTranslateOutput(false); - } - if (project.child(0) instanceof PhysicalNestedLoopJoin) { - ((PhysicalNestedLoopJoin) project.child(0)).setShouldTranslateOutput(false); + if (project.child(0) instanceof AbstractPhysicalJoin) { + ((AbstractPhysicalJoin) project.child(0)).setShouldTranslateOutput(false); } if (project.child(0) instanceof PhysicalFilter) { - if (project.child(0).child(0) instanceof PhysicalHashJoin) { - ((PhysicalHashJoin) project.child(0).child(0)).setShouldTranslateOutput(false); - } - if (project.child(0).child(0) instanceof PhysicalNestedLoopJoin) { - ((PhysicalNestedLoopJoin) project.child(0).child(0)).setShouldTranslateOutput(false); + if (project.child(0).child(0) instanceof AbstractPhysicalJoin) { + ((AbstractPhysicalJoin) project.child(0).child(0)).setShouldTranslateOutput(false); } } PlanFragment inputFragment = project.child(0).accept(this, context); @@ -1680,7 +1387,7 @@ public PlanFragment visitPhysicalProject(PhysicalProject project PlanNode inputPlanNode = inputFragment.getPlanRoot(); List slotList = project.getProjects() .stream() - .map(e -> e.toSlot()) + .map(NamedExpression::toSlot) .collect(Collectors.toList()); // For hash join node, use vSrcToOutputSMap to describe the expression calculation, use @@ -1710,7 +1417,7 @@ public PlanFragment visitPhysicalProject(PhysicalProject project if (inputPlanNode instanceof ScanNode) { TupleDescriptor tupleDescriptor = null; if (requiredByProjectSlotIdSet.size() != requiredSlotIdSet.size() - || execExprList.stream().collect(Collectors.toSet()).size() != execExprList.size() + || new HashSet<>(execExprList).size() != execExprList.size() || execExprList.stream().anyMatch(expr -> !(expr instanceof SlotRef))) { tupleDescriptor = generateTupleDesc(slotList, null, context); inputPlanNode.setProjectList(execExprList); @@ -1749,149 +1456,6 @@ public PlanFragment visitPhysicalProject(PhysicalProject project return inputFragment; } - private void updateChildSlotsMaterialization(PlanNode execPlan, - Set requiredSlotIdSet, Set requiredByProjectSlotIdSet, - PlanTranslatorContext context) { - Set slotRefSet = new HashSet<>(); - for (Expr expr : execPlan.getConjuncts()) { - expr.collect(SlotRef.class, slotRefSet); - } - Set slotIdSet = slotRefSet.stream() - .map(SlotRef::getSlotId).collect(Collectors.toSet()); - slotIdSet.addAll(requiredSlotIdSet); - boolean noneMaterialized = execPlan.getTupleIds().stream() - .map(context::getTupleDesc) - .map(TupleDescriptor::getSlots) - .flatMap(List::stream) - .peek(s -> s.setIsMaterialized(slotIdSet.contains(s.getId()))) - .filter(SlotDescriptor::isMaterialized) - .count() == 0; - if (noneMaterialized) { - context.getDescTable() - .getTupleDesc(execPlan.getTupleIds().get(0)).getSlots().get(0).setIsMaterialized(true); - } - if (execPlan instanceof ScanNode) { - try { - ((ScanNode) execPlan).updateRequiredSlots(context, requiredByProjectSlotIdSet); - } catch (UserException e) { - Util.logAndThrowRuntimeException(LOG, - "User Exception while reset external file scan node contexts.", e); - } - } - } - - @Override - public PlanFragment visitPhysicalFilter(PhysicalFilter filter, PlanTranslatorContext context) { - if (filter.child(0) instanceof AbstractPhysicalJoin) { - AbstractPhysicalJoin join = (AbstractPhysicalJoin) filter.child(0); - join.addFilterConjuncts(filter.getConjuncts()); - } - PlanFragment inputFragment = filter.child(0).accept(this, context); - - // Union contains oneRowRelation --> inputFragment = null - if (inputFragment == null) { - return inputFragment; - } - - PlanNode planNode = inputFragment.getPlanRoot(); - if (planNode instanceof ExchangeNode || planNode instanceof SortNode || planNode instanceof UnionNode) { - // the three nodes don't support conjuncts, need create a SelectNode to filter data - SelectNode selectNode = new SelectNode(context.nextPlanNodeId(), planNode); - addConjunctsToPlanNode(filter, selectNode, context); - addPlanRoot(inputFragment, selectNode, filter); - } else { - if (!(filter.child(0) instanceof AbstractPhysicalJoin)) { - addConjunctsToPlanNode(filter, planNode, context); - updateLegacyPlanIdToPhysicalPlan(inputFragment.getPlanRoot(), filter); - } - } - //in ut, filter.stats may be null - if (filter.getStats() != null) { - inputFragment.getPlanRoot().setCardinalityAfterFilter((long) filter.getStats().getRowCount()); - } - return inputFragment; - } - - private void addConjunctsToPlanNode(PhysicalFilter filter, - PlanNode planNode, - PlanTranslatorContext context) { - filter.getConjuncts().stream() - .map(e -> ExpressionTranslator.translate(e, context)) - .forEach(planNode::addConjunct); - updateLegacyPlanIdToPhysicalPlan(planNode, filter); - } - - @Override - public PlanFragment visitPhysicalLimit(PhysicalLimit physicalLimit, PlanTranslatorContext context) { - PlanFragment inputFragment = physicalLimit.child(0).accept(this, context); - - // Union contains oneRowRelation - if (inputFragment == null) { - return null; - } - - PlanNode child = inputFragment.getPlanRoot(); - if (physicalLimit.isGlobal()) { - if (child instanceof ExchangeNode) { - DataPartition outputPartition = DataPartition.UNPARTITIONED; - ExchangeNode exchangeNode = (ExchangeNode) inputFragment.getPlanRoot(); - inputFragment.setOutputPartition(outputPartition); - inputFragment.setPlanRoot(exchangeNode.getChild(0)); - inputFragment.setDestination(exchangeNode); - inputFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, DataPartition.UNPARTITIONED); - context.addPlanFragment(inputFragment); - } else if (physicalLimit.hasValidOffset()) { - // This case means GlobalLimit's child isn't gatherNode, which suggests the child is UNPARTITIONED - // When there is valid offset, exchangeNode should be added because other node don't support offset - inputFragment = createParentFragment(inputFragment, DataPartition.UNPARTITIONED, context); - child = inputFragment.getPlanRoot(); - } - } - child.setOffset(physicalLimit.getOffset()); - child.setLimit(physicalLimit.getLimit()); - updateLegacyPlanIdToPhysicalPlan(child, physicalLimit); - return inputFragment; - } - - @Override - public PlanFragment visitPhysicalDistribute(PhysicalDistribute distribute, - PlanTranslatorContext context) { - PlanFragment childFragment = distribute.child().accept(this, context); - - if (childFragment.getPlanRoot() instanceof AggregationNode - && distribute.child() instanceof PhysicalHashAggregate - && context.getFirstAggregateInFragment(childFragment) == distribute.child()) { - PhysicalHashAggregate hashAggregate = (PhysicalHashAggregate) distribute.child(); - if (hashAggregate.getAggPhase() == AggPhase.LOCAL - && hashAggregate.getAggMode() == AggMode.INPUT_TO_BUFFER) { - AggregationNode aggregationNode = (AggregationNode) childFragment.getPlanRoot(); - aggregationNode.setUseStreamingPreagg(hashAggregate.isMaybeUsingStream()); - } - } - - ExchangeNode exchange = new ExchangeNode(context.nextPlanNodeId(), childFragment.getPlanRoot()); - exchange.setNumInstances(childFragment.getPlanRoot().getNumInstances()); - childFragment.setPlanRoot(exchange); - updateLegacyPlanIdToPhysicalPlan(childFragment.getPlanRoot(), distribute); - return childFragment; - } - - @Override - public PlanFragment visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, - PlanTranslatorContext context) { - PlanFragment currentFragment = assertNumRows.child().accept(this, context); - // create assertNode - AssertNumRowsNode assertNumRowsNode = new AssertNumRowsNode(context.nextPlanNodeId(), - currentFragment.getPlanRoot(), - ExpressionTranslator.translateAssert(assertNumRows.getAssertNumRowsElement())); - if (currentFragment.getPlanRoot() instanceof ExchangeNode) { - currentFragment.setPlanRoot(currentFragment.getPlanRoot().getChild(0)); - currentFragment = createParentFragment(currentFragment, DataPartition.UNPARTITIONED, context); - } - addPlanRoot(currentFragment, assertNumRowsNode, assertNumRows); - return currentFragment; - } - /** * Returns a new fragment with a UnionNode as its root. The data partition of the * returned fragment and how the data of the child fragments is consumed depends on the @@ -1908,280 +1472,419 @@ public PlanFragment visitPhysicalAssertNumRows(PhysicalAssertNumRows childrenFragments = new ArrayList<>(); - Map childNodeToFragment = new HashMap<>(); for (Plan plan : setOperation.children()) { - PlanFragment planFragment = plan.accept(this, context); - if (planFragment != null) { - childrenFragments.add(planFragment); - } - childNodeToFragment.put(plan, planFragment); + childrenFragments.add(plan.accept(this, context)); } - PlanFragment setOperationFragment; - SetOperationNode setOperationNode; - - List allSlots = new Builder() - .addAll(setOperation.getOutput()) - .build(); - TupleDescriptor setTuple = generateTupleDesc(allSlots, null, context); - List outputSLotDescs = new ArrayList<>(setTuple.getSlots()); + TupleDescriptor setTuple = generateTupleDesc(setOperation.getOutput(), null, context); + List outputSlotDescs = new ArrayList<>(setTuple.getSlots()); + SetOperationNode setOperationNode; // create setOperationNode if (setOperation instanceof PhysicalUnion) { - setOperationNode = new UnionNode( - context.nextPlanNodeId(), setTuple.getId()); + setOperationNode = new UnionNode(context.nextPlanNodeId(), setTuple.getId()); } else if (setOperation instanceof PhysicalExcept) { - setOperationNode = new ExceptNode( - context.nextPlanNodeId(), setTuple.getId()); + setOperationNode = new ExceptNode(context.nextPlanNodeId(), setTuple.getId()); } else if (setOperation instanceof PhysicalIntersect) { - setOperationNode = new IntersectNode( - context.nextPlanNodeId(), setTuple.getId()); + setOperationNode = new IntersectNode(context.nextPlanNodeId(), setTuple.getId()); } else { - throw new RuntimeException("not support"); + throw new RuntimeException("not support set operation type " + setOperation); } - SetOperationResult setOperationResult = collectSetOperationResult(setOperation, childNodeToFragment); - for (List expressions : setOperationResult.getResultExpressions()) { - List resultExprs = expressions - .stream() - .map(expr -> ExpressionTranslator.translate(expr, context)) - .collect(ImmutableList.toImmutableList()); - setOperationNode.addResultExprLists(resultExprs); + setOperation.children().stream() + .map(Plan::getOutput) + .map(l -> l.stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(ImmutableList.toImmutableList())) + .forEach(setOperationNode::addResultExprLists); + if (setOperation instanceof PhysicalUnion) { + ((PhysicalUnion) setOperation).getConstantExprsList().stream() + .map(l -> l.stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(ImmutableList.toImmutableList())) + .forEach(setOperationNode::addConstExprList); } - for (List expressions : setOperationResult.getConstExpressions()) { - List constExprs = expressions - .stream() - .map(expr -> ExpressionTranslator.translate(expr, context)) - .collect(ImmutableList.toImmutableList()); - setOperationNode.addConstExprList(constExprs); + for (PlanFragment childFragment : childrenFragments) { + setOperationNode.addChild(childFragment.getPlanRoot()); } + setOperationNode.finalizeForNereids(outputSlotDescs, outputSlotDescs); - for (PlanFragment childFragment : childrenFragments) { - if (childFragment != null) { - setOperationNode.addChild(childFragment.getPlanRoot()); + PlanFragment setOperationFragment; + if (childrenFragments.isEmpty()) { + setOperationFragment = createPlanFragment(setOperationNode, + DataPartition.UNPARTITIONED, setOperation); + context.addPlanFragment(setOperationFragment); + } else { + int childrenSize = childrenFragments.size(); + setOperationFragment = childrenFragments.get(childrenSize - 1); + for (int i = childrenSize - 2; i >= 0; i--) { + context.removePlanFragment(childrenFragments.get(i)); + for (PlanFragment child : childrenFragments.get(i).getChildren()) { + setOperationFragment.addChild(child); + } + } + setPlanRoot(setOperationFragment, setOperationNode, setOperation); + } + + return setOperationFragment; + } + + /*- + * Physical sort: + * 1. Build sortInfo + * There are two types of slotRef: + * one is generated by the previous node, collectively called old. + * the other is newly generated by the sort node, collectively called new. + * Filling of sortInfo related data structures, + * a. ordering use newSlotRef. + * b. sortTupleSlotExprs use oldSlotRef. + * 2. Create sortNode + * 3. Create mergeFragment + * TODO: When the slotRef of sort is currently generated, + * it will be based on the expression in select and orderBy expression in to ensure the uniqueness of slotRef. + * But eg: + * select a+1 from table order by a+1; + * the expressions of the two are inconsistent. + * The former will perform an additional Alias. + * Currently we cannot test whether this will have any effect. + * After a+1 can be parsed , reprocessing. + */ + @Override + public PlanFragment visitPhysicalQuickSort(PhysicalQuickSort sort, + PlanTranslatorContext context) { + PlanFragment inputFragment = sort.child(0).accept(this, context); + + // 2. According to the type of sort, generate physical plan + if (!sort.getSortPhase().isMerge()) { + // For localSort or Gather->Sort, we just need to add sortNode + SortNode sortNode = translateSortNode(sort, inputFragment.getPlanRoot(), context); + addPlanRoot(inputFragment, sortNode, sort); + } else { + // For mergeSort, we need to push sortInfo to exchangeNode + if (!(inputFragment.getPlanRoot() instanceof ExchangeNode)) { + // if there is no exchange node for mergeSort + // e.g., localSort -> mergeSort + // It means the local has satisfied the Gather property. We can just ignore mergeSort + return inputFragment; } + SortNode sortNode = (SortNode) inputFragment.getPlanRoot().getChild(0); + ((ExchangeNode) inputFragment.getPlanRoot()).setMergeInfo(sortNode.getSortInfo()); } - setOperationNode.finalizeForNereids(outputSLotDescs, outputSLotDescs); - - // create setOperationFragment - // If all child fragments are unpartitioned, return a single unpartitioned fragment - // with a UnionNode that merges all child fragments. - if (allChildFragmentsUnPartitioned(childrenFragments)) { - setOperationFragment = createPlanFragment(setOperationNode, DataPartition.UNPARTITIONED, setOperation); - // Absorb the plan trees of all childFragments into unionNode - // and fix up the fragment tree in the process. - for (int i = 0; i < childrenFragments.size(); ++i) { - connectChildFragmentNotCheckExchangeNode(setOperationNode, i, setOperationFragment, - childrenFragments.get(i), - context); + return inputFragment; + } + + @Override + public PlanFragment visitPhysicalTopN(PhysicalTopN topN, PlanTranslatorContext context) { + PlanFragment inputFragment = topN.child(0).accept(this, context); + + // 2. According to the type of sort, generate physical plan + if (!topN.getSortPhase().isMerge()) { + // For localSort or Gather->Sort, we just need to add TopNNode + SortNode sortNode = translateSortNode(topN, inputFragment.getPlanRoot(), context); + sortNode.setOffset(topN.getOffset()); + sortNode.setLimit(topN.getLimit()); + if (topN.getMutableState(PhysicalTopN.TOPN_RUNTIME_FILTER).isPresent()) { + sortNode.setUseTopnOpt(true); + PlanNode child = sortNode.getChild(0); + Preconditions.checkArgument(child instanceof OlapScanNode, + "topN opt expect OlapScanNode, but we get " + child); + OlapScanNode scanNode = ((OlapScanNode) child); + scanNode.setUseTopnOpt(true); + } + // push sort to scan opt + if (sortNode.getChild(0) instanceof OlapScanNode) { + OlapScanNode scanNode = ((OlapScanNode) sortNode.getChild(0)); + if (checkPushSort(sortNode, scanNode.getOlapTable())) { + SortInfo sortInfo = sortNode.getSortInfo(); + scanNode.setSortInfo(sortInfo); + scanNode.getSortInfo().setSortTupleSlotExprs(sortNode.getResolvedTupleExprs()); + for (Expr expr : sortInfo.getOrderingExprs()) { + scanNode.getSortInfo().addMaterializedOrderingExpr(expr); + } + if (sortNode.getOffset() > 0) { + scanNode.setSortLimit(sortNode.getLimit() + sortNode.getOffset()); + } else { + scanNode.setSortLimit(sortNode.getLimit()); + } + } } + addPlanRoot(inputFragment, sortNode, topN); } else { - setOperationFragment = createPlanFragment(setOperationNode, - new DataPartition(TPartitionType.HASH_PARTITIONED, - setOperationNode.getMaterializedResultExprLists().get(0)), setOperation); - for (int i = 0; i < childrenFragments.size(); ++i) { - PlanFragment childFragment = childrenFragments.get(i); - // Connect the unpartitioned child fragments to SetOperationNode via a random exchange. - connectChildFragmentNotCheckExchangeNode( - setOperationNode, i, setOperationFragment, childFragment, context); - childFragment.setOutputPartition( - DataPartition.hashPartitioned(setOperationNode.getMaterializedResultExprLists().get(i))); + // For mergeSort, we need to push sortInfo to exchangeNode + if (!(inputFragment.getPlanRoot() instanceof ExchangeNode)) { + // if there is no exchange node for mergeSort + // e.g., mergeTopN -> localTopN + // It means the local has satisfied the Gather property. We can just ignore mergeSort + inputFragment.getPlanRoot().setOffset(topN.getOffset()); + inputFragment.getPlanRoot().setLimit(topN.getLimit()); + return inputFragment; } + ExchangeNode exchangeNode = (ExchangeNode) inputFragment.getPlanRoot(); + exchangeNode.setMergeInfo(((SortNode) exchangeNode.getChild(0)).getSortInfo()); + exchangeNode.setLimit(topN.getLimit()); + exchangeNode.setOffset(topN.getOffset()); } - context.addPlanFragment(setOperationFragment); - return setOperationFragment; + updateLegacyPlanIdToPhysicalPlan(inputFragment.getPlanRoot(), topN); + return inputFragment; } @Override - public PlanFragment visitPhysicalGenerate(PhysicalGenerate generate, - PlanTranslatorContext context) { - PlanFragment currentFragment = generate.child().accept(this, context); - ArrayList functionCalls = generate.getGenerators().stream() - .map(e -> ExpressionTranslator.translate(e, context)) - .collect(Collectors.toCollection(ArrayList::new)); - TupleDescriptor tupleDescriptor = generateTupleDesc(generate.getGeneratorOutput(), null, context); - List childOutputTupleIds = currentFragment.getPlanRoot().getOutputTupleIds(); - if (childOutputTupleIds == null || childOutputTupleIds.isEmpty()) { - childOutputTupleIds = currentFragment.getPlanRoot().getTupleIds(); - } - List outputSlotIds = Stream.concat(childOutputTupleIds.stream(), - Stream.of(tupleDescriptor.getId())) - .map(id -> context.getTupleDesc(id).getSlots()) - .flatMap(List::stream) - .map(SlotDescriptor::getId) - .collect(Collectors.toList()); - TableFunctionNode tableFunctionNode = new TableFunctionNode(context.nextPlanNodeId(), - currentFragment.getPlanRoot(), tupleDescriptor.getId(), functionCalls, outputSlotIds); - addPlanRoot(currentFragment, tableFunctionNode, generate); - return currentFragment; - } + public PlanFragment visitPhysicalRepeat(PhysicalRepeat repeat, PlanTranslatorContext context) { + PlanFragment inputPlanFragment = repeat.child(0).accept(this, context); - @Override - public PlanFragment visitPhysicalCTEConsumer(PhysicalCTEConsumer cteConsumer, - PlanTranslatorContext context) { - CTEId cteId = cteConsumer.getCteId(); + Set sortedVirtualSlots = repeat.getSortedVirtualSlots(); + TupleDescriptor virtualSlotsTuple = + generateTupleDesc(ImmutableList.copyOf(sortedVirtualSlots), null, context); - MultiCastPlanFragment multCastFragment = (MultiCastPlanFragment) context.getCteProduceFragments().get(cteId); - Preconditions.checkState(multCastFragment.getSink() instanceof MultiCastDataSink, - "invalid multCastFragment"); + ImmutableSet flattenGroupingSetExprs = ImmutableSet.copyOf( + ExpressionUtils.flatExpressions(repeat.getGroupingSets())); - MultiCastDataSink multiCastDataSink = (MultiCastDataSink) multCastFragment.getSink(); - Preconditions.checkState(multiCastDataSink != null, "invalid multiCastDataSink"); + List aggregateFunctionUsedSlots = repeat.getOutputExpressions() + .stream() + .filter(output -> !(output instanceof VirtualSlotReference)) + .filter(output -> !flattenGroupingSetExprs.contains(output)) + .distinct() + .map(NamedExpression::toSlot) + .collect(ImmutableList.toImmutableList()); - PhysicalCTEProducer cteProducer = context.getCteProduceMap().get(cteId); - Preconditions.checkState(cteProducer != null, "invalid cteProducer"); + Set usedSlotInRepeat = ImmutableSet.builder() + .addAll(flattenGroupingSetExprs) + .addAll(aggregateFunctionUsedSlots) + .build(); - ExchangeNode exchangeNode = new ExchangeNode(context.nextPlanNodeId(), multCastFragment.getPlanRoot()); + List preRepeatExprs = usedSlotInRepeat.stream() + .map(expr -> ExpressionTranslator.translate(expr, context)) + .collect(ImmutableList.toImmutableList()); - DataStreamSink streamSink = new DataStreamSink(exchangeNode.getId()); - streamSink.setPartition(DataPartition.RANDOM); - streamSink.setFragment(multCastFragment); + List outputSlots = repeat.getOutputExpressions() + .stream() + .map(NamedExpression::toSlot) + .collect(ImmutableList.toImmutableList()); - multiCastDataSink.getDataStreamSinks().add(streamSink); - multiCastDataSink.getDestinations().add(Lists.newArrayList()); + // NOTE: we should first translate preRepeatExprs, then generate output tuple, + // or else the preRepeatExprs can not find the bottom slotRef and throw + // exception: invalid slot id + TupleDescriptor outputTuple = generateTupleDesc(outputSlots, null, context); - exchangeNode.setNumInstances(multCastFragment.getPlanRoot().getNumInstances()); + // cube and rollup already convert to grouping sets in LogicalPlanBuilder.withAggregate() + GroupingInfo groupingInfo = new GroupingInfo( + GroupingType.GROUPING_SETS, virtualSlotsTuple, outputTuple, preRepeatExprs); - PlanFragment consumeFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, - multCastFragment.getDataPartition()); + List> repeatSlotIdList = repeat.computeRepeatSlotIdList(getSlotIds(outputTuple)); + Set allSlotId = repeatSlotIdList.stream() + .flatMap(Set::stream) + .collect(ImmutableSet.toImmutableSet()); - Map projectMap = Maps.newHashMap(); - projectMap.putAll(cteConsumer.getProducerToConsumerSlotMap()); + RepeatNode repeatNode = new RepeatNode(context.nextPlanNodeId(), + inputPlanFragment.getPlanRoot(), groupingInfo, repeatSlotIdList, + allSlotId, repeat.computeVirtualSlotValues(sortedVirtualSlots)); + repeatNode.setNumInstances(inputPlanFragment.getPlanRoot().getNumInstances()); + addPlanRoot(inputPlanFragment, repeatNode, repeat); + updateLegacyPlanIdToPhysicalPlan(inputPlanFragment.getPlanRoot(), repeat); + return inputPlanFragment; + } - List execList = new ArrayList<>(); - PlanNode inputPlanNode = consumeFragment.getPlanRoot(); - List cteProjects = cteProducer.getProjects(); - for (Slot slot : cteProjects) { - if (projectMap.containsKey(slot)) { - execList.add(projectMap.get(slot)); - } else { - throw new RuntimeException("could not find slot in cte producer consumer projectMap"); - } - } + @Override + public PlanFragment visitPhysicalWindow(PhysicalWindow physicalWindow, + PlanTranslatorContext context) { + PlanFragment inputPlanFragment = physicalWindow.child(0).accept(this, context); - List slotList = execList - .stream() - .map(e -> e.toSlot()) - .collect(Collectors.toList()); + // 1. translate to old optimizer variable + // variable in Nereids + WindowFrameGroup windowFrameGroup = physicalWindow.getWindowFrameGroup(); + List partitionKeyList = Lists.newArrayList(windowFrameGroup.getPartitionKeys()); + List orderKeyList = windowFrameGroup.getOrderKeys(); + List windowFunctionList = windowFrameGroup.getGroups(); + WindowFrame windowFrame = windowFrameGroup.getWindowFrame(); - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, null, context); + // partition by clause + List partitionExprs = partitionKeyList.stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(Collectors.toList()); - // update tuple list and tblTupleList - inputPlanNode.getTupleIds().clear(); - inputPlanNode.getTupleIds().add(tupleDescriptor.getId()); - inputPlanNode.getTblRefIds().clear(); - inputPlanNode.getTblRefIds().add(tupleDescriptor.getId()); - inputPlanNode.getNullableTupleIds().clear(); - inputPlanNode.getNullableTupleIds().add(tupleDescriptor.getId()); + // order by clause + List orderByElements = orderKeyList.stream() + .map(orderKey -> new OrderByElement( + ExpressionTranslator.translate(orderKey.child(), context), + orderKey.isAsc(), orderKey.isNullFirst())) + .collect(Collectors.toList()); - List execExprList = execList - .stream() - .map(e -> ExpressionTranslator.translate(e, context)) + // function calls + List analyticFnCalls = windowFunctionList.stream() + .map(e -> { + Expression function = e.child(0).child(0); + if (function instanceof AggregateFunction) { + AggregateParam param = AggregateParam.localResult(); + function = new AggregateExpression((AggregateFunction) function, param); + } + return ExpressionTranslator.translate(function, context); + }) + .map(FunctionCallExpr.class::cast) + .peek(fnCall -> { + fnCall.setIsAnalyticFnCall(true); + ((org.apache.doris.catalog.AggregateFunction) fnCall.getFn()).setIsAnalyticFn(true); + }) .collect(Collectors.toList()); - inputPlanNode.setProjectList(execExprList); - inputPlanNode.setOutputTupleDesc(tupleDescriptor); + // analytic window + AnalyticWindow analyticWindow = physicalWindow.translateWindowFrame(windowFrame, context); - // update data partition - DataPartition dataPartition = new DataPartition(TPartitionType.HASH_PARTITIONED, execExprList); - consumeFragment.setDataPartition(dataPartition); + // 2. get bufferedTupleDesc from SortNode and compute isNullableMatched + Map bufferedSlotRefForWindow = getBufferedSlotRefForWindow(windowFrameGroup, context); + TupleDescriptor bufferedTupleDesc = context.getBufferedTupleForWindow(); - SelectNode projectNode = new SelectNode(context.nextPlanNodeId(), inputPlanNode); - consumeFragment.setPlanRoot(projectNode); + // generate predicates to check if the exprs of partitionKeys and orderKeys have matched isNullable between + // sortNode and analyticNode + Expr partitionExprsIsNullableMatched = partitionExprs.isEmpty() ? null : windowExprsHaveMatchedNullable( + partitionKeyList, partitionExprs, bufferedSlotRefForWindow); - multCastFragment.getDestNodeList().add(exchangeNode); - consumeFragment.addChild(multCastFragment); - context.getPlanFragments().add(consumeFragment); + Expr orderElementsIsNullableMatched = orderByElements.isEmpty() ? null : windowExprsHaveMatchedNullable( + orderKeyList.stream().map(UnaryNode::child).collect(Collectors.toList()), + orderByElements.stream().map(OrderByElement::getExpr).collect(Collectors.toList()), + bufferedSlotRefForWindow); - return consumeFragment; + // 3. generate tupleDesc + List windowSlotList = windowFunctionList.stream() + .map(NamedExpression::toSlot) + .collect(Collectors.toList()); + TupleDescriptor outputTupleDesc = generateTupleDesc(windowSlotList, null, context); + + // 4. generate AnalyticEvalNode + AnalyticEvalNode analyticEvalNode = new AnalyticEvalNode( + context.nextPlanNodeId(), + inputPlanFragment.getPlanRoot(), + analyticFnCalls, + partitionExprs, + orderByElements, + analyticWindow, + outputTupleDesc, + outputTupleDesc, + partitionExprsIsNullableMatched, + orderElementsIsNullableMatched, + bufferedTupleDesc + ); + analyticEvalNode.setNumInstances(inputPlanFragment.getPlanRoot().getNumInstances()); + inputPlanFragment.addPlanRoot(analyticEvalNode); + return inputPlanFragment; } - @Override - public PlanFragment visitPhysicalCTEProducer(PhysicalCTEProducer cteProducer, - PlanTranslatorContext context) { - PlanFragment child = cteProducer.child().accept(this, context); - CTEId cteId = cteProducer.getCteId(); - context.getPlanFragments().remove(child); - MultiCastPlanFragment cteProduce = new MultiCastPlanFragment(child); - MultiCastDataSink multiCastDataSink = new MultiCastDataSink(); - cteProduce.setSink(multiCastDataSink); + /* ******************************************************************************************** + * private functions + * ******************************************************************************************** */ - List outputs = cteProducer.getProjects().stream() + private PartitionSortNode translatePartitionSortNode(PhysicalPartitionTopN partitionTopN, + PlanNode childNode, PlanTranslatorContext context) { + // Generate the SortInfo, similar to 'translateSortNode'. + List oldOrderingExprList = Lists.newArrayList(); + List ascOrderList = Lists.newArrayList(); + List nullsFirstParamList = Lists.newArrayList(); + List orderKeyList = partitionTopN.getOrderKeys(); + // 1. Get previous slotRef + orderKeyList.forEach(k -> { + oldOrderingExprList.add(ExpressionTranslator.translate(k.getExpr(), context)); + ascOrderList.add(k.isAsc()); + nullsFirstParamList.add(k.isNullFirst()); + }); + List sortTupleOutputList = new ArrayList<>(); + List outputList = partitionTopN.getOutput(); + outputList.forEach(k -> sortTupleOutputList.add(ExpressionTranslator.translate(k, context))); + List partitionExprs = partitionTopN.getPartitionKeys().stream() .map(e -> ExpressionTranslator.translate(e, context)) .collect(Collectors.toList()); + // 2. Generate new Tuple and get current slotRef for newOrderingExprList + List newOrderingExprList = Lists.newArrayList(); + TupleDescriptor tupleDesc = generateTupleDesc(outputList, orderKeyList, newOrderingExprList, context, null); + // 3. fill in SortInfo members + SortInfo sortInfo = new SortInfo(newOrderingExprList, ascOrderList, nullsFirstParamList, tupleDesc); - cteProduce.setOutputExprs(outputs); - context.getCteProduceFragments().put(cteId, cteProduce); - context.getCteProduceMap().put(cteId, cteProducer); - context.getPlanFragments().add(cteProduce); - return child; - } + PartitionSortNode partitionSortNode = new PartitionSortNode(context.nextPlanNodeId(), childNode, + partitionTopN.getFunction(), partitionExprs, sortInfo, partitionTopN.hasGlobalLimit(), + partitionTopN.getPartitionLimit(), sortTupleOutputList, oldOrderingExprList); - /** - * NOTICE: Must translate left, which it's the producer of consumer. - */ - @Override - public PlanFragment visitPhysicalCTEAnchor(PhysicalCTEAnchor cteAnchor, - PlanTranslatorContext context) { - cteAnchor.child(0).accept(this, context); - return cteAnchor.child(1).accept(this, context); + if (partitionTopN.getStats() != null) { + partitionSortNode.setCardinality((long) partitionTopN.getStats().getRowCount()); + } + updateLegacyPlanIdToPhysicalPlan(partitionSortNode, partitionTopN); + return partitionSortNode; } - private List castCommonDataTypeOutputs(List outputs, List childOutputs) { - List newChildOutputs = new ArrayList<>(); - for (int i = 0; i < outputs.size(); ++i) { - Slot right = childOutputs.get(i); - DataType tightestCommonType = outputs.get(i).getDataType(); - Expression newRight = TypeCoercionUtils.castIfNotMatchType(right, tightestCommonType); - newChildOutputs.add(newRight); + private SortNode translateSortNode(AbstractPhysicalSort sort, PlanNode childNode, + PlanTranslatorContext context) { + List oldOrderingExprList = Lists.newArrayList(); + List ascOrderList = Lists.newArrayList(); + List nullsFirstParamList = Lists.newArrayList(); + List orderKeyList = sort.getOrderKeys(); + // 1. Get previous slotRef + orderKeyList.forEach(k -> { + oldOrderingExprList.add(ExpressionTranslator.translate(k.getExpr(), context)); + ascOrderList.add(k.isAsc()); + nullsFirstParamList.add(k.isNullFirst()); + }); + List sortTupleOutputList = new ArrayList<>(); + List outputList = sort.getOutput(); + outputList.forEach(k -> sortTupleOutputList.add(ExpressionTranslator.translate(k, context))); + // 2. Generate new Tuple and get current slotRef for newOrderingExprList + List newOrderingExprList = Lists.newArrayList(); + TupleDescriptor tupleDesc = generateTupleDesc(outputList, orderKeyList, newOrderingExprList, context, null); + // 3. fill in SortInfo members + SortInfo sortInfo = new SortInfo(newOrderingExprList, ascOrderList, nullsFirstParamList, tupleDesc); + SortNode sortNode = new SortNode(context.nextPlanNodeId(), childNode, sortInfo, true); + sortNode.finalizeForNereids(tupleDesc, sortTupleOutputList, oldOrderingExprList); + if (sort.getMutableState(PhysicalTopN.TWO_PHASE_READ_OPT).isPresent()) { + sortNode.setUseTwoPhaseReadOpt(true); + sortNode.getSortInfo().setUseTwoPhaseRead(); + injectRowIdColumnSlot(sortNode.getSortInfo().getSortTupleDescriptor()); + TupleDescriptor childTuple = childNode.getOutputTupleDesc() != null + ? childNode.getOutputTupleDesc() : context.getTupleDesc(childNode.getTupleIds().get(0)); + SlotDescriptor childRowIdDesc = childTuple.getSlots().get(childTuple.getSlots().size() - 1); + sortNode.getResolvedTupleExprs().add(new SlotRef(childRowIdDesc)); } - return ImmutableList.copyOf(newChildOutputs); - } - - private SetOperationResult collectSetOperationResult( - PhysicalSetOperation setOperation, Map childPlanToFragment) { - List> resultExprs = new ArrayList<>(); - List> constExprs = new ArrayList<>(); - List outputs = setOperation.getOutput(); - for (Plan child : setOperation.children()) { - List castCommonDataTypeOutputs = castCommonDataTypeOutputs(outputs, child.getOutput()); - if (child.anyMatch(PhysicalOneRowRelation.class::isInstance) && childPlanToFragment.get(child) == null) { - constExprs.add(collectConstExpressions(castCommonDataTypeOutputs, child)); - } else { - resultExprs.add(castCommonDataTypeOutputs); - } + if (sort.getStats() != null) { + sortNode.setCardinality((long) sort.getStats().getRowCount()); } - return new SetOperationResult(resultExprs, constExprs); + updateLegacyPlanIdToPhysicalPlan(sortNode, sort); + return sortNode; } - private List collectConstExpressions( - List castExpressions, Plan child) { - List newCastExpressions = new ArrayList<>(); - for (int i = 0; i < castExpressions.size(); ++i) { - Expression expression = castExpressions.get(i); - if (expression instanceof Cast) { - newCastExpressions.add(expression.withChildren( - (collectPhysicalOneRowRelation(child).getProjects().get(i).children()))); - } else { - newCastExpressions.add( - (collectPhysicalOneRowRelation(child).getProjects().get(i))); + private void updateChildSlotsMaterialization(PlanNode execPlan, + Set requiredSlotIdSet, Set requiredByProjectSlotIdSet, + PlanTranslatorContext context) { + Set slotRefSet = new HashSet<>(); + for (Expr expr : execPlan.getConjuncts()) { + expr.collect(SlotRef.class, slotRefSet); + } + Set slotIdSet = slotRefSet.stream() + .map(SlotRef::getSlotId).collect(Collectors.toSet()); + slotIdSet.addAll(requiredSlotIdSet); + boolean noneMaterialized = execPlan.getTupleIds().stream() + .map(context::getTupleDesc) + .map(TupleDescriptor::getSlots) + .flatMap(List::stream) + .peek(s -> s.setIsMaterialized(slotIdSet.contains(s.getId()))) + .filter(SlotDescriptor::isMaterialized) + .count() == 0; + if (noneMaterialized) { + context.getDescTable() + .getTupleDesc(execPlan.getTupleIds().get(0)).getSlots().get(0).setIsMaterialized(true); + } + if (execPlan instanceof ScanNode) { + try { + ((ScanNode) execPlan).updateRequiredSlots(context, requiredByProjectSlotIdSet); + } catch (UserException e) { + Util.logAndThrowRuntimeException(LOG, + "User Exception while reset external file scan node contexts.", e); } } - return newCastExpressions; - } - - private PhysicalOneRowRelation collectPhysicalOneRowRelation(Plan child) { - return (PhysicalOneRowRelation) - ((ImmutableSet) child.collect(PhysicalOneRowRelation.class::isInstance)).asList().get(0); } - private boolean allChildFragmentsUnPartitioned(List childrenFragments) { - boolean allChildFragmentsUnPartitioned = true; - for (PlanFragment child : childrenFragments) { - allChildFragmentsUnPartitioned = allChildFragmentsUnPartitioned && !child.isPartitioned(); - } - return allChildFragmentsUnPartitioned; + private void addConjunctsToPlanNode(PhysicalFilter filter, + PlanNode planNode, + PlanTranslatorContext context) { + filter.getConjuncts().stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .forEach(planNode::addConjunct); + updateLegacyPlanIdToPhysicalPlan(planNode, filter); } private void extractExecSlot(Expr root, Set slotIdList) { @@ -2249,260 +1952,48 @@ private TupleDescriptor generateTupleDesc(List slotList, List or return tupleDescriptor; } - private PlanFragment createParentFragment(PlanFragment childFragment, DataPartition parentPartition, - PlanTranslatorContext context) { - ExchangeNode exchangeNode = new ExchangeNode(context.nextPlanNodeId(), childFragment.getPlanRoot()); - exchangeNode.setNumInstances(childFragment.getPlanRoot().getNumInstances()); - PlanFragment parentFragment = new PlanFragment(context.nextFragmentId(), exchangeNode, parentPartition); - childFragment.setDestination(exchangeNode); - childFragment.setOutputPartition(parentPartition); - context.addPlanFragment(parentFragment); - return parentFragment; - } - - private void connectChildFragment(PlanNode parent, int childIdx, - PlanFragment parentFragment, PlanFragment childFragment, - PlanTranslatorContext context) { - PlanNode exchange = parent.getChild(childIdx); - if (!(exchange instanceof ExchangeNode)) { - exchange = new ExchangeNode(context.nextPlanNodeId(), childFragment.getPlanRoot()); - exchange.setNumInstances(childFragment.getPlanRoot().getNumInstances()); - } - childFragment.setPlanRoot(exchange.getChild(0)); - exchange.setFragment(parentFragment); - parent.setChild(childIdx, exchange); - childFragment.setDestination((ExchangeNode) exchange); - } - - private void connectChildFragmentNotCheckExchangeNode(PlanNode parent, int childIdx, - PlanFragment parentFragment, PlanFragment childFragment, - PlanTranslatorContext context) { - PlanNode exchange = new ExchangeNode(context.nextPlanNodeId(), childFragment.getPlanRoot()); - exchange.setNumInstances(childFragment.getPlanRoot().getNumInstances()); - childFragment.setPlanRoot(exchange.getChild(0)); - exchange.setFragment(parentFragment); - parent.setChild(childIdx, exchange); - childFragment.setDestination((ExchangeNode) exchange); - } - - /** - * Return unpartitioned fragment that merges the input fragment's output via - * an ExchangeNode. - * Requires that input fragment be partitioned. - */ - private PlanFragment exchangeToMergeFragment(PlanFragment inputFragment, PlanTranslatorContext context) { - if (!inputFragment.isPartitioned()) { - return inputFragment; - } - - // exchange node clones the behavior of its input, aside from the conjuncts - ExchangeNode mergePlan = new ExchangeNode(context.nextPlanNodeId(), inputFragment.getPlanRoot()); - DataPartition dataPartition = DataPartition.UNPARTITIONED; - mergePlan.setNumInstances(inputFragment.getPlanRoot().getNumInstances()); - PlanFragment fragment = new PlanFragment(context.nextFragmentId(), mergePlan, dataPartition); - inputFragment.setDestination(mergePlan); - context.addPlanFragment(fragment); - return fragment; - } - - private PlanFragment constructColocateJoin(HashJoinNode hashJoinNode, PlanFragment leftFragment, + private PlanFragment connectJoinNode(HashJoinNode hashJoinNode, PlanFragment leftFragment, PlanFragment rightFragment, PlanTranslatorContext context, AbstractPlan join) { - // TODO: add reason - hashJoinNode.setColocate(true, ""); hashJoinNode.setChild(0, leftFragment.getPlanRoot()); hashJoinNode.setChild(1, rightFragment.getPlanRoot()); setPlanRoot(leftFragment, hashJoinNode, join); rightFragment.getTargetRuntimeFilterIds().forEach(leftFragment::setTargetRuntimeFilterIds); rightFragment.getBuilderRuntimeFilterIds().forEach(leftFragment::setBuilderRuntimeFilterIds); context.removePlanFragment(rightFragment); - leftFragment.setHasColocatePlanNode(true); - return leftFragment; - } - - private PlanFragment constructBucketShuffleJoin(PhysicalHashJoin physicalHashJoin, - HashJoinNode hashJoinNode, PlanFragment leftFragment, - PlanFragment rightFragment, PlanTranslatorContext context) { - // according to left partition to generate right partition expr list - DistributionSpecHash leftDistributionSpec - = (DistributionSpecHash) physicalHashJoin.left().getPhysicalProperties().getDistributionSpec(); - - Pair, List> onClauseUsedSlots = physicalHashJoin.getHashConjunctsExprIds(); - List rightPartitionExprIds = Lists.newArrayList(leftDistributionSpec.getOrderedShuffledColumns()); - for (int i = 0; i < leftDistributionSpec.getOrderedShuffledColumns().size(); i++) { - int idx = leftDistributionSpec.getExprIdToEquivalenceSet() - .get(leftDistributionSpec.getOrderedShuffledColumns().get(i)); - ExprId leftShuffleColumnId = leftDistributionSpec.getOrderedShuffledColumns().get(i); - Set equivalIds = leftDistributionSpec.getEquivalenceExprIdsOf(leftShuffleColumnId); - int index = -1; - for (ExprId id : equivalIds) { - index = onClauseUsedSlots.first.indexOf(id); - if (index != -1) { - break; - } - } - Preconditions.checkArgument(index != -1); - rightPartitionExprIds.set(idx, onClauseUsedSlots.second.get(index)); - } - // assemble fragment - hashJoinNode.setDistributionMode(HashJoinNode.DistributionMode.BUCKET_SHUFFLE); - if (leftDistributionSpec.getShuffleType() != ShuffleType.NATURAL - && leftDistributionSpec.getShuffleType() != ShuffleType.BUCKETED) { - hashJoinNode.setDistributionMode(DistributionMode.PARTITIONED); - } - connectChildFragment(hashJoinNode, 1, leftFragment, rightFragment, context); - setPlanRoot(leftFragment, hashJoinNode, physicalHashJoin); - // HASH_PARTITIONED and BUCKET_SHFFULE_HASH_PARTITIONED are two type of hash algorithm - // And the nature left child means it use BUCKET_SHFFULE_HASH_PARTITIONED in storage layer - TPartitionType partitionType = TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED; - if (leftDistributionSpec.getShuffleType() != ShuffleType.NATURAL) { - partitionType = TPartitionType.HASH_PARTITIONED; + for (PlanFragment rightChild : rightFragment.getChildren()) { + leftFragment.addChild(rightChild); } - DataPartition rhsJoinPartition = new DataPartition(partitionType, - rightPartitionExprIds.stream().map(context::findSlotRef).collect(Collectors.toList())); - rightFragment.setOutputPartition(rhsJoinPartition); - - return leftFragment; - } - - private PlanFragment constructBroadcastJoin(HashJoinNode hashJoinNode, PlanFragment leftFragment, - PlanFragment rightFragment, PlanTranslatorContext context, AbstractPlan join) { - hashJoinNode.setDistributionMode(DistributionMode.BROADCAST); - setPlanRoot(leftFragment, hashJoinNode, join); - rightFragment.setRightChildOfBroadcastHashJoin(true); - connectChildFragment(hashJoinNode, 1, leftFragment, rightFragment, context); return leftFragment; } - private PlanFragment constructShuffleJoin(AbstractPhysicalJoin physicalHashJoin, - HashJoinNode hashJoinNode, PlanFragment leftFragment, - PlanFragment rightFragment, PlanTranslatorContext context) { - hashJoinNode.setDistributionMode(HashJoinNode.DistributionMode.PARTITIONED); - // TODO should according nereids distribute indicate - // first, extract join exprs - List eqJoinConjuncts = hashJoinNode.getEqJoinConjuncts(); - List lhsJoinExprs = Lists.newArrayList(); - List rhsJoinExprs = Lists.newArrayList(); - for (BinaryPredicate eqJoinPredicate : eqJoinConjuncts) { - // no remapping necessary - lhsJoinExprs.add(eqJoinPredicate.getChild(0).clone(null)); - rhsJoinExprs.add(eqJoinPredicate.getChild(1).clone(null)); - } - - // create the parent fragment containing the HashJoin node - DataPartition lhsJoinPartition = new DataPartition(TPartitionType.HASH_PARTITIONED, - Expr.cloneList(lhsJoinExprs, null)); - DataPartition rhsJoinPartition = - new DataPartition(TPartitionType.HASH_PARTITIONED, rhsJoinExprs); - PlanFragment joinFragment = createPlanFragment(hashJoinNode, lhsJoinPartition, physicalHashJoin); - context.addPlanFragment(joinFragment); - - connectChildFragment(hashJoinNode, 0, joinFragment, leftFragment, context); - connectChildFragment(hashJoinNode, 1, joinFragment, rightFragment, context); - - leftFragment.setOutputPartition(lhsJoinPartition); - rightFragment.setOutputPartition(rhsJoinPartition); - - return joinFragment; - } - - private List collectGroupBySlots(List groupByExpressionList, - List outputExpressionList) { - List groupSlotList = Lists.newArrayList(); - Set virtualSlotReferences = groupByExpressionList.stream() + private List collectGroupBySlots(List groupByExpressions, + List outputExpressions) { + List groupSlots = Lists.newArrayList(); + Set virtualSlotReferences = groupByExpressions.stream() .filter(VirtualSlotReference.class::isInstance) .map(VirtualSlotReference.class::cast) .collect(Collectors.toSet()); - for (Expression e : groupByExpressionList) { - if (e instanceof SlotReference && outputExpressionList.stream().anyMatch(o -> o.anyMatch(e::equals))) { - groupSlotList.add((SlotReference) e); + for (Expression e : groupByExpressions) { + if (e instanceof SlotReference && outputExpressions.stream().anyMatch(o -> o.anyMatch(e::equals))) { + groupSlots.add((SlotReference) e); } else if (e instanceof SlotReference && !virtualSlotReferences.isEmpty()) { // When there is a virtualSlot, it is a groupingSets scenario, // and the original exprId should be retained at this time. - groupSlotList.add((SlotReference) e); + groupSlots.add((SlotReference) e); } else { - groupSlotList.add(new SlotReference(e.toSql(), e.getDataType(), e.nullable(), ImmutableList.of())); + groupSlots.add(new SlotReference(e.toSql(), e.getDataType(), e.nullable(), ImmutableList.of())); } } - return groupSlotList; + return groupSlots; } - private List getSlotIdList(TupleDescriptor tupleDescriptor) { + private List getSlotIds(TupleDescriptor tupleDescriptor) { return tupleDescriptor.getSlots() .stream() .map(slot -> slot.getId().asInt()) .collect(ImmutableList.toImmutableList()); } - private boolean isUnnecessaryProject(PhysicalProject project) { - // The project list for agg is always needed,since tuple of agg contains the slots used by group by expr - return !hasPrune(project) && !hasExprCalc(project); - } - - private boolean hasPrune(PhysicalProject project) { - PhysicalPlan child = (PhysicalPlan) project.child(0); - - return project.getProjects().size() != child.getOutput().size(); - } - - private boolean isFragmentPartitioned(PlanFragment fragment) { - return fragment.isPartitioned() && fragment.getPlanRoot().getNumInstances() > 1; - } - - private boolean hasExprCalc(PhysicalProject project) { - for (NamedExpression p : project.getProjects()) { - if (p.children().size() > 1) { - return true; - } - for (Expression e : p.children()) { - if (!(e instanceof SlotReference)) { - return true; - } - } - } - return false; - } - - private List removeAlias(PhysicalProject project) { - List namedExpressions = project.getProjects(); - List slotReferences = new ArrayList<>(); - for (NamedExpression n : namedExpressions) { - if (n instanceof Alias) { - slotReferences.add((SlotReference) n.child(0)); - } else { - slotReferences.add((SlotReference) n); - } - } - return slotReferences; - } - - private Optional toDataPartition(PhysicalDistribute distribute, - Optional> partitionExpressions, PlanTranslatorContext context) { - if (distribute.getDistributionSpec() == DistributionSpecGather.INSTANCE) { - return Optional.of(DataPartition.UNPARTITIONED); - } else if (distribute.getDistributionSpec() == DistributionSpecReplicated.INSTANCE) { - // the data partition should be left child of join - return Optional.empty(); - } else if (distribute.getDistributionSpec() instanceof DistributionSpecHash - || distribute.getDistributionSpec() == DistributionSpecAny.INSTANCE) { - if (!partitionExpressions.isPresent()) { - throw new AnalysisException("Missing partition expressions"); - } - Preconditions.checkState( - partitionExpressions.get().stream().allMatch(expr -> expr instanceof SlotReference), - "All partition expression should be slot: " + partitionExpressions.get()); - if (!partitionExpressions.isPresent() || partitionExpressions.get().isEmpty()) { - return Optional.of(DataPartition.UNPARTITIONED); - } - List partitionExprs = partitionExpressions.get() - .stream() - .map(p -> ExpressionTranslator.translate(p, context)) - .collect(ImmutableList.toImmutableList()); - return Optional.of(new DataPartition(TPartitionType.HASH_PARTITIONED, partitionExprs)); - } else { - return Optional.empty(); - } - } - private Map getBufferedSlotRefForWindow(WindowFrameGroup windowFrameGroup, PlanTranslatorContext context) { Map bufferedSlotRefForWindow = context.getBufferedSlotRefForWindow(); @@ -2515,7 +2006,7 @@ private Map getBufferedSlotRefForWindow(WindowFrameGroup window bufferedSlotRefForWindow.putIfAbsent(exprId, context.findSlotRef(exprId)); }); windowFrameGroup.getOrderKeys().stream() - .map(ok -> ok.child()) + .map(UnaryNode::child) .map(NamedExpression.class::cast) .forEach(expression -> { ExprId exprId = expression.getExprId(); @@ -2557,56 +2048,78 @@ private Expr windowExprsHaveMatchedNullable(Map exprIdToExpr, Map< new CompoundPredicate(CompoundPredicate.Operator.OR, bothNull, lhsEqRhsNotNull), remainder); } - private DataPartition hashSpecToDataPartition(PhysicalDistribute distribute, PlanTranslatorContext context) { - Preconditions.checkState(distribute.getDistributionSpec() instanceof DistributionSpecHash); - DistributionSpecHash hashSpec = (DistributionSpecHash) distribute.getDistributionSpec(); - List partitions = hashSpec.getOrderedShuffledColumns().stream() - .map(exprId -> context.findSlotRef(exprId)) - .collect(Collectors.toList()); - return new DataPartition(TPartitionType.HASH_PARTITIONED, partitions); - } - - private static class SetOperationResult { - private final List> resultExpressions; - private final List> constExpressions; - - public SetOperationResult(List> resultExpressions, List> constExpressions) { - this.resultExpressions = ImmutableList.copyOf(resultExpressions); - this.constExpressions = ImmutableList.copyOf(constExpressions); - } - - public List> getConstExpressions() { - return constExpressions; - } - - public List> getResultExpressions() { - return resultExpressions; - } - } - private PlanFragment createPlanFragment(PlanNode planNode, DataPartition dataPartition, AbstractPlan physicalPlan) { PlanFragment planFragment = new PlanFragment(context.nextFragmentId(), planNode, dataPartition); updateLegacyPlanIdToPhysicalPlan(planNode, physicalPlan); return planFragment; } + // TODO: refactor this, call it every where is not a good way private void setPlanRoot(PlanFragment fragment, PlanNode planNode, AbstractPlan physicalPlan) { fragment.setPlanRoot(planNode); updateLegacyPlanIdToPhysicalPlan(planNode, physicalPlan); } + // TODO: refactor this, call it every where is not a good way private void addPlanRoot(PlanFragment fragment, PlanNode planNode, AbstractPlan physicalPlan) { fragment.addPlanRoot(planNode); updateLegacyPlanIdToPhysicalPlan(planNode, physicalPlan); } + private DataPartition toDataPartition(DistributionSpec distributionSpec, + List childOutputIds, PlanTranslatorContext context) { + if (distributionSpec instanceof DistributionSpecAny + || distributionSpec instanceof DistributionSpecStorageAny + || distributionSpec instanceof DistributionSpecExecutionAny) { + return DataPartition.RANDOM; + } else if (distributionSpec instanceof DistributionSpecGather + || distributionSpec instanceof DistributionSpecStorageGather + || distributionSpec instanceof DistributionSpecReplicated) { + return DataPartition.UNPARTITIONED; + } else if (distributionSpec instanceof DistributionSpecHash) { + DistributionSpecHash distributionSpecHash = (DistributionSpecHash) distributionSpec; + List partitionExprs = Lists.newArrayList(); + for (int i = 0; i < distributionSpecHash.getEquivalenceExprIds().size(); i++) { + Set equivalenceExprId = distributionSpecHash.getEquivalenceExprIds().get(i); + for (ExprId exprId : equivalenceExprId) { + if (childOutputIds.contains(exprId)) { + partitionExprs.add(context.findSlotRef(exprId)); + break; + } + } + if (partitionExprs.size() != i + 1) { + throw new RuntimeException("Cannot translate DistributionSpec to DataPartition," + + " DistributionSpec: " + distributionSpec + + ", child output: " + childOutputIds); + } + } + TPartitionType partitionType; + switch (distributionSpecHash.getShuffleType()) { + case STORAGE_BUCKETED: + partitionType = TPartitionType.BUCKET_SHFFULE_HASH_PARTITIONED; + break; + case EXECUTION_BUCKETED: + partitionType = TPartitionType.HASH_PARTITIONED; + break; + case NATURAL: + default: + throw new RuntimeException("Do not support shuffle type: " + + distributionSpecHash.getShuffleType()); + } + return new DataPartition(partitionType, partitionExprs); + } else { + throw new RuntimeException("Unknown DistributionSpec: " + distributionSpec); + } + } + + // TODO: refactor this, call it every where is not a good way private void updateLegacyPlanIdToPhysicalPlan(PlanNode planNode, AbstractPlan physicalPlan) { if (statsErrorEstimator != null) { statsErrorEstimator.updateLegacyPlanIdToPhysicalPlan(planNode, physicalPlan); } } - private SlotDescriptor injectRowIdColumnSlot(TupleDescriptor tupleDesc) { + private void injectRowIdColumnSlot(TupleDescriptor tupleDesc) { SlotDescriptor slotDesc = context.addSlotDesc(tupleDesc); LOG.debug("inject slot {}", slotDesc); String name = Column.ROWID_COL; @@ -2615,6 +2128,109 @@ private SlotDescriptor injectRowIdColumnSlot(TupleDescriptor tupleDesc) { slotDesc.setColumn(col); slotDesc.setIsNullable(false); slotDesc.setIsMaterialized(true); - return slotDesc; + } + + /** + * We use two phase read to optimize sql like: select * from tbl [where xxx = ???] [order by column1] [limit n] + * in the first phase, we add an extra column `RowId` to Block, and sort blocks in TopN nodes + * in the second phase, we have n rows, we do a fetch rpc to get all rowids data for the n rows + * and reconstruct the final block + */ + private void setResultSinkFetchOptionIfNeed() { + boolean needFetch = false; + // Only single olap table should be fetched + OlapTable fetchOlapTable = null; + OlapScanNode scanNode = null; + for (PlanFragment fragment : context.getPlanFragments()) { + PlanNode node = fragment.getPlanRoot(); + PlanNode parent = null; + // OlapScanNode is the last node. + // So, just get the last two node and check if they are SortNode and OlapScan. + while (node.getChildren().size() != 0) { + parent = node; + node = node.getChildren().get(0); + } + + // case1: general topn optimized query + if ((node instanceof OlapScanNode) && (parent instanceof SortNode)) { + SortNode sortNode = (SortNode) parent; + scanNode = (OlapScanNode) node; + if (sortNode.getUseTwoPhaseReadOpt()) { + needFetch = true; + fetchOlapTable = scanNode.getOlapTable(); + break; + } + } + } + for (PlanFragment fragment : context.getPlanFragments()) { + if (needFetch && fragment.getSink() instanceof ResultSink) { + TFetchOption fetchOption = new TFetchOption(); + fetchOption.setFetchRowStore(fetchOlapTable.storeRowColumn()); + fetchOption.setUseTwoPhaseFetch(true); + fetchOption.setNodesInfo(SystemInfoService.createAliveNodesInfo()); + if (!fetchOlapTable.storeRowColumn()) { + // Set column desc for each column + List columnsDesc = new ArrayList<>(); + scanNode.getColumnDesc(columnsDesc, null, null); + fetchOption.setColumnDesc(columnsDesc); + } + ((ResultSink) fragment.getSink()).setFetchOption(fetchOption); + break; + } + } + } + + /** + * topN opt: using storage data ordering to accelerate topn operation. + * refer pr: optimize topn query if order by columns is prefix of sort keys of table (#10694) + */ + private boolean checkPushSort(SortNode sortNode, OlapTable olapTable) { + // Ensure limit is less then threshold + if (sortNode.getLimit() <= 0 + || sortNode.getLimit() > ConnectContext.get().getSessionVariable().topnOptLimitThreshold) { + return false; + } + + // Ensure all isAscOrder is same, ande length != 0. + // Can't be zorder. + if (sortNode.getSortInfo().getIsAscOrder().stream().distinct().count() != 1 + || olapTable.isZOrderSort()) { + return false; + } + + // Tablet's order by key only can be the front part of schema. + // Like: schema: a.b.c.d.e.f.g order by key: a.b.c (no a,b,d) + // Do **prefix match** to check if order by key can be pushed down. + // olap order by key: a.b.c.d + // sort key: (a) (a,b) (a,b,c) (a,b,c,d) is ok + // (a,c) (a,c,d), (a,c,b) (a,c,f) (a,b,c,d,e)is NOT ok + List sortExprs = sortNode.getSortInfo().getOrderingExprs(); + List nullsFirsts = sortNode.getSortInfo().getNullsFirst(); + List isAscOrders = sortNode.getSortInfo().getIsAscOrder(); + if (sortExprs.size() > olapTable.getDataSortInfo().getColNum()) { + return false; + } + for (int i = 0; i < sortExprs.size(); i++) { + // table key. + Column tableKey = olapTable.getFullSchema().get(i); + // sort slot. + Expr sortExpr = sortExprs.get(i); + if (sortExpr instanceof SlotRef) { + SlotRef slotRef = (SlotRef) sortExpr; + if (tableKey.equals(slotRef.getColumn())) { + // ORDER BY DESC NULLS FIRST can not be optimized to only read file tail, + // since NULLS is at file head but data is at tail + if (tableKey.isAllowNull() && nullsFirsts.get(i) && !isAscOrders.get(i)) { + return false; + } + } else { + return false; + } + } else { + return false; + } + } + + return true; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/CostAndEnforcerJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/CostAndEnforcerJob.java index b673bd6e875631..f6ae1d6879bd31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/CostAndEnforcerJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/CostAndEnforcerJob.java @@ -222,8 +222,8 @@ private boolean calculateEnforce(List requestChildrenPropert // it's certain that lowestCostChildren is equals to arity(). ChildrenPropertiesRegulator regulator = new ChildrenPropertiesRegulator(groupExpression, lowestCostChildren, outputChildrenProperties, requestChildrenProperties, context); - double enforceCost = regulator.adjustChildrenProperties(); - if (enforceCost < 0) { + boolean success = regulator.adjustChildrenProperties(); + if (!success) { // invalid enforce, return. return false; } 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 8fb6296fd43c22..4d76b77cd19da7 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 @@ -35,7 +35,6 @@ import org.apache.doris.nereids.rules.analysis.ResolveOrdinalInOrderByAndGroupBy; import org.apache.doris.nereids.rules.analysis.SubqueryToApply; import org.apache.doris.nereids.rules.analysis.UserAuthentication; -import org.apache.doris.nereids.rules.rewrite.HideOneRowRelationUnderUnion; import java.util.List; @@ -70,8 +69,7 @@ public class Analyzer extends AbstractBatchJobExecutor { // please see rule BindSlotReference or BindFunction for example new ProjectWithDistinctToAggregate(), new ResolveOrdinalInOrderByAndGroupBy(), - new ReplaceExpressionByChildOutput(), - new HideOneRowRelationUnderUnion() + new ReplaceExpressionByChildOutput() ), topDown( new FillUpMissingSlots(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java index 70d7ed2e8e48b8..f7bd7c1a5a45a9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/executor/Rewriter.java @@ -63,6 +63,7 @@ import org.apache.doris.nereids.rules.rewrite.InferPredicates; import org.apache.doris.nereids.rules.rewrite.InlineCTE; import org.apache.doris.nereids.rules.rewrite.MergeFilters; +import org.apache.doris.nereids.rules.rewrite.MergeOneRowRelationIntoUnion; import org.apache.doris.nereids.rules.rewrite.MergeProjects; import org.apache.doris.nereids.rules.rewrite.MergeSetOperations; import org.apache.doris.nereids.rules.rewrite.NormalizeAggregate; @@ -71,6 +72,8 @@ import org.apache.doris.nereids.rules.rewrite.PruneOlapScanPartition; import org.apache.doris.nereids.rules.rewrite.PruneOlapScanTablet; import org.apache.doris.nereids.rules.rewrite.PushFilterInsideJoin; +import org.apache.doris.nereids.rules.rewrite.PushProjectIntoOneRowRelation; +import org.apache.doris.nereids.rules.rewrite.PushProjectThroughUnion; import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughProject; import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughWindow; import org.apache.doris.nereids.rules.rewrite.PushdownLimit; @@ -223,10 +226,13 @@ public class Rewriter extends AbstractBatchJobExecutor { // this rule should invoke after ColumnPruning custom(RuleType.ELIMINATE_UNNECESSARY_PROJECT, EliminateUnnecessaryProject::new), - topic("Intersection optimization", + topic("Set operation optimization", // Do MergeSetOperation first because we hope to match pattern of Distinct SetOperator. + topDown(new PushProjectThroughUnion(), new MergeProjects()), bottomUp(new MergeSetOperations()), - bottomUp(new BuildAggForUnion()) + bottomUp(new PushProjectIntoOneRowRelation()), + topDown(new MergeOneRowRelationIntoUnion()), + topDown(new BuildAggForUnion()) ), topic("Window optimization", diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java index 008ed0e8173a86..6264dd2ce66728 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/GroupExpression.java @@ -248,9 +248,14 @@ public Cost getCostValueByProperties(PhysicalProperties property) { return lowestCostTable.get(property).first; } - public void putOutputPropertiesMap(PhysicalProperties outputPropertySet, - PhysicalProperties requiredPropertySet) { - this.requestPropertiesMap.put(requiredPropertySet, outputPropertySet); + public void putOutputPropertiesMap(PhysicalProperties outputProperties, + PhysicalProperties requiredProperties) { + this.requestPropertiesMap.put(requiredProperties, outputProperties); + } + + public void putOutputPropertiesMapIfAbsent(PhysicalProperties outputProperties, + PhysicalProperties requiredProperties) { + this.requestPropertiesMap.putIfAbsent(requiredProperties, outputProperties); } /** diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterGenerator.java index 9ec1a34c9e0838..d042d9a08369e5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterGenerator.java @@ -142,6 +142,9 @@ public PhysicalPlan visitPhysicalHashJoin(PhysicalHashJoin olapTableSink, + PlanContext context) { + return PhysicalProperties.GATHER; } @Override - public PhysicalProperties visitPhysicalCTEProducer( - PhysicalCTEProducer cteProducer, PlanContext context) { - Preconditions.checkState(childrenOutputProperties.size() == 1); - return childrenOutputProperties.get(0); + public PhysicalProperties visit(Plan plan, PlanContext context) { + return PhysicalProperties.ANY; } + /* ******************************************************************************************** + * Leaf Plan Node, in lexicographical order + * ******************************************************************************************** */ + @Override public PhysicalProperties visitPhysicalCTEConsumer( PhysicalCTEConsumer cteConsumer, PlanContext context) { @@ -106,13 +118,45 @@ public PhysicalProperties visitPhysicalCTEConsumer( } @Override - public PhysicalProperties visitPhysicalCTEAnchor( - PhysicalCTEAnchor cteAnchor, PlanContext context) { - Preconditions.checkState(childrenOutputProperties.size() == 2); - // return properties inherited from consumer side which may further be used at upper layer - return childrenOutputProperties.get(1); + public PhysicalProperties visitPhysicalEmptyRelation(PhysicalEmptyRelation emptyRelation, PlanContext context) { + return PhysicalProperties.GATHER; + } + + @Override + public PhysicalProperties visitPhysicalEsScan(PhysicalEsScan esScan, PlanContext context) { + return PhysicalProperties.ANY; + } + + @Override + public PhysicalProperties visitPhysicalFileScan(PhysicalFileScan fileScan, PlanContext context) { + return PhysicalProperties.ANY; + } + + @Override + public PhysicalProperties visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, PlanContext context) { + return PhysicalProperties.ANY; } + @Override + public PhysicalProperties visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanContext context) { + return new PhysicalProperties(olapScan.getDistributionSpec()); + } + + @Override + public PhysicalProperties visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, PlanContext context) { + return PhysicalProperties.GATHER; + } + + @Override + public PhysicalProperties visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, PlanContext context) { + TableValuedFunction function = tvfRelation.getFunction(); + return function.getPhysicalProperties(); + } + + /* ******************************************************************************************** + * Other Node, in lexicographical order + * ******************************************************************************************** */ + @Override public PhysicalProperties visitPhysicalHashAggregate( PhysicalHashAggregate agg, PlanContext context) { @@ -123,13 +167,6 @@ public PhysicalProperties visitPhysicalHashAggregate( case GLOBAL: case DISTINCT_LOCAL: case DISTINCT_GLOBAL: - DistributionSpec childSpec = childOutputProperty.getDistributionSpec(); - // If child's property is enforced, change it to bucketed - if (childSpec instanceof DistributionSpecHash - && ((DistributionSpecHash) childSpec).getShuffleType().equals(ShuffleType.ENFORCED)) { - DistributionSpecHash distributionSpecHash = (DistributionSpecHash) childSpec; - return new PhysicalProperties(distributionSpecHash.withShuffleType(ShuffleType.BUCKETED)); - } return new PhysicalProperties(childOutputProperty.getDistributionSpec()); default: throw new RuntimeException("Could not derive output properties for agg phase: " + agg.getAggPhase()); @@ -137,74 +174,32 @@ public PhysicalProperties visitPhysicalHashAggregate( } @Override - public PhysicalProperties visitAbstractPhysicalSort(AbstractPhysicalSort sort, + public PhysicalProperties visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, PlanContext context) { Preconditions.checkState(childrenOutputProperties.size() == 1); - if (sort.getSortPhase().isLocal()) { - return new PhysicalProperties( - childrenOutputProperties.get(0).getDistributionSpec(), - new OrderSpec(sort.getOrderKeys())); - } - return new PhysicalProperties(DistributionSpecGather.INSTANCE, new OrderSpec(sort.getOrderKeys())); + PhysicalProperties childOutputProperty = childrenOutputProperties.get(0); + return new PhysicalProperties(childOutputProperty.getDistributionSpec()); } @Override - public PhysicalProperties visitPhysicalLimit(PhysicalLimit limit, PlanContext context) { - Preconditions.checkState(childrenOutputProperties.size() == 1); - PhysicalProperties childOutputProperty = childrenOutputProperties.get(0); - if (limit.getPhase().isLocal()) { - return new PhysicalProperties(childOutputProperty.getDistributionSpec(), - childOutputProperty.getOrderSpec()); - } - return new PhysicalProperties(DistributionSpecGather.INSTANCE, childOutputProperty.getOrderSpec()); + public PhysicalProperties visitPhysicalCTEAnchor( + PhysicalCTEAnchor cteAnchor, PlanContext context) { + Preconditions.checkState(childrenOutputProperties.size() == 2); + // return properties inherited from consumer side which may further be used at upper layer + return childrenOutputProperties.get(1); } @Override - public PhysicalProperties visitPhysicalPartitionTopN(PhysicalPartitionTopN partitionTopN, - PlanContext context) { + public PhysicalProperties visitPhysicalCTEProducer( + PhysicalCTEProducer cteProducer, PlanContext context) { Preconditions.checkState(childrenOutputProperties.size() == 1); - PhysicalProperties childOutputProperty = childrenOutputProperties.get(0); - return new PhysicalProperties(childOutputProperty.getDistributionSpec()); + return childrenOutputProperties.get(0); } @Override - public PhysicalProperties visitPhysicalProject(PhysicalProject project, PlanContext context) { - // TODO: order spec do not process since we do not use it. - Preconditions.checkState(childrenOutputProperties.size() == 1); - PhysicalProperties childProperties = childrenOutputProperties.get(0); - DistributionSpec childDistributionSpec = childProperties.getDistributionSpec(); - OrderSpec childOrderSpec = childProperties.getOrderSpec(); - DistributionSpec outputDistributionSpec; - if (childDistributionSpec instanceof DistributionSpecHash) { - Map projections = Maps.newHashMap(); - Set obstructions = Sets.newHashSet(); - for (NamedExpression namedExpression : project.getProjects()) { - if (namedExpression instanceof Alias) { - Alias alias = (Alias) namedExpression; - Expression child = alias.child(); - if (child instanceof SlotReference) { - projections.put(((SlotReference) child).getExprId(), alias.getExprId()); - } else if (child instanceof Cast && child.child(0) instanceof Slot - && isSameHashValue(child.child(0).getDataType(), child.getDataType())) { - // cast(slot as varchar(10)) can do projection if slot is varchar(3) - projections.put(((Slot) child.child(0)).getExprId(), alias.getExprId()); - } else { - obstructions.addAll( - child.getInputSlots().stream() - .map(NamedExpression::getExprId) - .collect(Collectors.toSet())); - } - } - } - if (projections.entrySet().stream().allMatch(kv -> kv.getKey().equals(kv.getValue()))) { - return childrenOutputProperties.get(0); - } - outputDistributionSpec = ((DistributionSpecHash) childDistributionSpec).project(projections, obstructions); - return new PhysicalProperties(outputDistributionSpec, childOrderSpec); - } else { - return childrenOutputProperties.get(0); - } - + public PhysicalProperties visitPhysicalDistribute( + PhysicalDistribute distribute, PlanContext context) { + return distribute.getPhysicalProperties(); } @Override @@ -214,9 +209,9 @@ public PhysicalProperties visitPhysicalFilter(PhysicalFilter fil } @Override - public PhysicalProperties visitPhysicalDistribute( - PhysicalDistribute distribute, PlanContext context) { - return distribute.getPhysicalProperties(); + public PhysicalProperties visitPhysicalGenerate(PhysicalGenerate generate, PlanContext context) { + Preconditions.checkState(childrenOutputProperties.size() == 1); + return childrenOutputProperties.get(0); } @Override @@ -248,87 +243,159 @@ public PhysicalProperties visitPhysicalHashJoin( // shuffle, if left child is natural mean current join is bucket shuffle join // and remain natural for colocate join on upper join. - return new PhysicalProperties(DistributionSpecHash.merge(leftHashSpec, rightHashSpec, - leftHashSpec.getShuffleType() == ShuffleType.NATURAL ? ShuffleType.NATURAL : ShuffleType.BUCKETED)); + return new PhysicalProperties(DistributionSpecHash.merge( + leftHashSpec, rightHashSpec, leftHashSpec.getShuffleType())); } throw new RuntimeException("Could not derive hash join's output properties. join: " + hashJoin); } + @Override + public PhysicalProperties visitPhysicalLimit(PhysicalLimit limit, PlanContext context) { + Preconditions.checkState(childrenOutputProperties.size() == 1); + return childrenOutputProperties.get(0); + } + @Override public PhysicalProperties visitPhysicalNestedLoopJoin( PhysicalNestedLoopJoin nestedLoopJoin, PlanContext context) { - // TODO: currently, only support cross join in BE Preconditions.checkState(childrenOutputProperties.size() == 2); PhysicalProperties leftOutputProperty = childrenOutputProperties.get(0); return new PhysicalProperties(leftOutputProperty.getDistributionSpec()); } @Override - public PhysicalProperties visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanContext context) { - // TODO: find a better way to handle both tablet num == 1 and colocate table together in future - if (!olapScan.getTable().isColocateTable() && olapScan.getScanTabletNum() == 1 - && (!ConnectContext.get().getSessionVariable().enablePipelineEngine() - || ConnectContext.get().getSessionVariable().getParallelExecInstanceNum() == 1)) { - return PhysicalProperties.GATHER; - } else if (olapScan.getDistributionSpec() instanceof DistributionSpecHash) { - return PhysicalProperties.createHash((DistributionSpecHash) olapScan.getDistributionSpec()); + public PhysicalProperties visitPhysicalProject(PhysicalProject project, PlanContext context) { + // TODO: order spec do not process since we do not use it. + Preconditions.checkState(childrenOutputProperties.size() == 1); + PhysicalProperties childProperties = childrenOutputProperties.get(0); + DistributionSpec childDistributionSpec = childProperties.getDistributionSpec(); + OrderSpec childOrderSpec = childProperties.getOrderSpec(); + if (childDistributionSpec instanceof DistributionSpecHash) { + Map projections = Maps.newHashMap(); + Set obstructions = Sets.newHashSet(); + for (NamedExpression namedExpression : project.getProjects()) { + if (namedExpression instanceof Alias) { + Alias alias = (Alias) namedExpression; + Expression child = alias.child(); + if (child instanceof SlotReference) { + projections.put(((SlotReference) child).getExprId(), alias.getExprId()); + } else if (child instanceof Cast && child.child(0) instanceof Slot + && isSameHashValue(child.child(0).getDataType(), child.getDataType())) { + // cast(slot as varchar(10)) can do projection if slot is varchar(3) + projections.put(((Slot) child.child(0)).getExprId(), alias.getExprId()); + } else { + obstructions.addAll( + child.getInputSlots().stream() + .map(NamedExpression::getExprId) + .collect(Collectors.toSet())); + } + } + } + if (projections.entrySet().stream().allMatch(kv -> kv.getKey().equals(kv.getValue()))) { + return childrenOutputProperties.get(0); + } + DistributionSpecHash childDistributionSpecHash = (DistributionSpecHash) childDistributionSpec; + DistributionSpec defaultAnySpec = childDistributionSpecHash.getShuffleType() == ShuffleType.NATURAL + ? DistributionSpecStorageAny.INSTANCE : DistributionSpecAny.INSTANCE; + DistributionSpec outputDistributionSpec = childDistributionSpecHash.project( + projections, obstructions, defaultAnySpec); + return new PhysicalProperties(outputDistributionSpec, childOrderSpec); } else { - return PhysicalProperties.ANY; + return childrenOutputProperties.get(0); } } @Override - public PhysicalProperties visitPhysicalFileScan(PhysicalFileScan fileScan, PlanContext context) { - return PhysicalProperties.ANY; - } - - @Override - public PhysicalProperties visitPhysicalEmptyRelation(PhysicalEmptyRelation emptyRelation, PlanContext context) { - return PhysicalProperties.GATHER; - } - - @Override - public PhysicalProperties visitPhysicalStorageLayerAggregate( - PhysicalStorageLayerAggregate storageLayerAggregate, PlanContext context) { - return storageLayerAggregate.getRelation().accept(this, context); + public PhysicalProperties visitPhysicalRepeat(PhysicalRepeat repeat, PlanContext context) { + Preconditions.checkState(childrenOutputProperties.size() == 1); + return PhysicalProperties.ANY.withOrderSpec(childrenOutputProperties.get(0).getOrderSpec()); } @Override - public PhysicalProperties visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, PlanContext context) { - return PhysicalProperties.ANY; + public PhysicalProperties visitPhysicalPartitionTopN(PhysicalPartitionTopN partitionTopN, + PlanContext context) { + Preconditions.checkState(childrenOutputProperties.size() == 1); + PhysicalProperties childOutputProperty = childrenOutputProperties.get(0); + return new PhysicalProperties(childOutputProperty.getDistributionSpec()); } @Override - public PhysicalProperties visitPhysicalEsScan(PhysicalEsScan esScan, PlanContext context) { - return PhysicalProperties.ANY; + public PhysicalProperties visitPhysicalSetOperation(PhysicalSetOperation setOperation, PlanContext context) { + int[] offsetsOfFirstChild = null; + ShuffleType firstType = null; + List childrenDistribution = childrenOutputProperties.stream() + .map(PhysicalProperties::getDistributionSpec) + .collect(Collectors.toList()); + if (childrenDistribution.isEmpty()) { + // no child, mean it only has some one-row-relations + return PhysicalProperties.GATHER; + } + if (childrenDistribution.stream().allMatch(DistributionSpecGather.class::isInstance)) { + return PhysicalProperties.GATHER; + } + for (int i = 0; i < childrenDistribution.size(); i++) { + DistributionSpec childDistribution = childrenDistribution.get(i); + if (!(childDistribution instanceof DistributionSpecHash)) { + return PhysicalProperties.ANY; + } + DistributionSpecHash distributionSpecHash = (DistributionSpecHash) childDistribution; + int[] offsetsOfCurrentChild = new int[distributionSpecHash.getOrderedShuffledColumns().size()]; + for (int j = 0; j < setOperation.getChildOutput(i).size(); j++) { + int offset = distributionSpecHash.getExprIdToEquivalenceSet() + .getOrDefault(setOperation.getChildOutput(i).get(j).getExprId(), -1); + if (offset > 0) { + offsetsOfCurrentChild[offset] = j; + } else { + return PhysicalProperties.ANY; + } + } + if (offsetsOfFirstChild == null) { + firstType = ((DistributionSpecHash) childDistribution).getShuffleType(); + offsetsOfFirstChild = offsetsOfCurrentChild; + } else if (!Arrays.equals(offsetsOfFirstChild, offsetsOfCurrentChild) + || firstType != ((DistributionSpecHash) childDistribution).getShuffleType()) { + return PhysicalProperties.ANY; + } + } + // bucket + List request = Lists.newArrayList(); + for (int offset : offsetsOfFirstChild) { + request.add(setOperation.getOutput().get(offset).getExprId()); + } + return PhysicalProperties.createHash(request, firstType); } @Override - public PhysicalProperties visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, PlanContext context) { - TableValuedFunction function = tvfRelation.getFunction(); - return function.getPhysicalProperties(); + public PhysicalProperties visitPhysicalUnion(PhysicalUnion union, PlanContext context) { + if (union.getConstantExprsList().isEmpty()) { + return visitPhysicalSetOperation(union, context); + } else { + // current be could not run const expr on appropriate node, + // so if we have constant exprs on union, the output of union always any + return PhysicalProperties.ANY; + } } @Override - public PhysicalProperties visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, + public PhysicalProperties visitAbstractPhysicalSort(AbstractPhysicalSort sort, PlanContext context) { - return PhysicalProperties.GATHER; + Preconditions.checkState(childrenOutputProperties.size() == 1); + if (sort.getSortPhase().isLocal()) { + return new PhysicalProperties( + childrenOutputProperties.get(0).getDistributionSpec(), + new OrderSpec(sort.getOrderKeys())); + } + return new PhysicalProperties(DistributionSpecGather.INSTANCE, new OrderSpec(sort.getOrderKeys())); } @Override - public PhysicalProperties visitPhysicalGenerate(PhysicalGenerate generate, PlanContext context) { + public PhysicalProperties visitPhysicalWindow(PhysicalWindow window, PlanContext context) { Preconditions.checkState(childrenOutputProperties.size() == 1); return childrenOutputProperties.get(0); } - @Override - public PhysicalProperties visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, - PlanContext context) { - return PhysicalProperties.GATHER; - } - private boolean isSameHashValue(DataType originType, DataType castType) { if (originType.isStringLikeType() && (castType.isVarcharType() || castType.isStringType()) && (castType.width() >= originType.width() || castType.width() < 0)) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java index bd73a33cf1059b..3f811a23625d67 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildrenPropertiesRegulator.java @@ -29,28 +29,29 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalNestedLoopJoin; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.JoinUtils; -import org.apache.doris.qe.ConnectContext; import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import java.util.ArrayList; import java.util.List; +import java.util.Optional; import java.util.Set; /** - * ensure child add enough distribute. + * ensure child add enough distribute. update children properties if we do regular */ -public class ChildrenPropertiesRegulator extends PlanVisitor { +public class ChildrenPropertiesRegulator extends PlanVisitor { private final GroupExpression parent; private final List children; private final List childrenProperties; private final List requiredProperties; private final JobContext jobContext; - private double enforceCost = 0.0; public ChildrenPropertiesRegulator(GroupExpression parent, List children, List childrenProperties, List requiredProperties, @@ -67,26 +68,27 @@ public ChildrenPropertiesRegulator(GroupExpression parent, List * * @return enforce cost. */ - public double adjustChildrenProperties() { + public boolean adjustChildrenProperties() { return parent.getPlan().accept(this, null); } @Override - public Double visit(Plan plan, Void context) { - return enforceCost; + public Boolean visit(Plan plan, Void context) { + return true; } @Override - public Double visitPhysicalHashAggregate(PhysicalHashAggregate agg, Void context) { + public Boolean visitPhysicalHashAggregate(PhysicalHashAggregate agg, Void context) { if (agg.getAggMode() == AggMode.INPUT_TO_RESULT && children.get(0).getPlan() instanceof PhysicalDistribute) { - return -1.0; + // this means one stage gather agg, usually bad pattern + return false; } - return 0.0; + return true; } @Override - public Double visitPhysicalHashJoin(PhysicalHashJoin hashJoin, + public Boolean visitPhysicalHashJoin(PhysicalHashJoin hashJoin, Void context) { Preconditions.checkArgument(children.size() == 2, String.format("children.size() is %d", children.size())); Preconditions.checkArgument(childrenProperties.size() == 2); @@ -96,106 +98,248 @@ public Double visitPhysicalHashJoin(PhysicalHashJoin> leftLowest - = leftChild.getLowestCostTable().get(childrenProperties.get(0)); - PhysicalProperties leftOutput = leftChild.getOutputProperties(childrenProperties.get(0)); + Optional updatedForLeft = Optional.empty(); + Optional updatedForRight = Optional.empty(); - GroupExpression rightChild = children.get(1); - Pair> rightLowest - = rightChild.getLowestCostTable().get(childrenProperties.get(1)); - PhysicalProperties rightOutput = rightChild.getOutputProperties(childrenProperties.get(1)); - - // check colocate join - if (leftHashSpec.getShuffleType() == ShuffleType.NATURAL - && rightHashSpec.getShuffleType() == ShuffleType.NATURAL) { + if ((leftHashSpec.getShuffleType() == ShuffleType.NATURAL + && rightHashSpec.getShuffleType() == ShuffleType.NATURAL)) { + // check colocate join with scan if (JoinUtils.couldColocateJoin(leftHashSpec, rightHashSpec)) { - return enforceCost; + return true; + } + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.STORAGE_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); + } else if (leftHashSpec.getShuffleType() == ShuffleType.NATURAL + && rightHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED) { + // must add enforce because shuffle algorithm is not same between NATURAL and BUCKETED + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.STORAGE_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); + } else if (leftHashSpec.getShuffleType() == ShuffleType.NATURAL + && rightHashSpec.getShuffleType() == ShuffleType.STORAGE_BUCKETED) { + if (bothSideShuffleKeysAreSameOrder(leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())) { + return true; + } + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.STORAGE_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); + } else if (leftHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED + && rightHashSpec.getShuffleType() == ShuffleType.NATURAL) { + // TODO: we must do shuffle on right because coordinator could not do right be selection in this case, + // since it always to check the left most node whether olap scan node. + // after we fix coordinator problem, we could do right to left bucket shuffle + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.EXECUTION_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); + } else if (leftHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED + && rightHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED) { + if (bothSideShuffleKeysAreSameOrder(rightHashSpec, leftHashSpec, + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec())) { + return true; + } + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.EXECUTION_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); + } else if ((leftHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED + && rightHashSpec.getShuffleType() == ShuffleType.STORAGE_BUCKETED)) { + if (children.get(0).getPlan() instanceof PhysicalDistribute) { + updatedForLeft = Optional.of(calAnotherSideRequired( + ShuffleType.STORAGE_BUCKETED, rightHashSpec, leftHashSpec, + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec())); + } else { + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.EXECUTION_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); + } + } else if ((leftHashSpec.getShuffleType() == ShuffleType.STORAGE_BUCKETED + && rightHashSpec.getShuffleType() == ShuffleType.NATURAL)) { + // TODO: we must do shuffle on right because coordinator could not do right be selection in this case, + // since it always to check the left most node whether olap scan node. + // after we fix coordinator problem, we could do right to left bucket shuffle + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.STORAGE_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); + } else if ((leftHashSpec.getShuffleType() == ShuffleType.STORAGE_BUCKETED + && rightHashSpec.getShuffleType() == ShuffleType.EXECUTION_BUCKETED)) { + if (children.get(0).getPlan() instanceof PhysicalDistribute) { + updatedForLeft = Optional.of(calAnotherSideRequired( + ShuffleType.EXECUTION_BUCKETED, rightHashSpec, leftHashSpec, + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec())); + } else { + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.STORAGE_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); } - } - // check bucket shuffle join - if (leftHashSpec.getShuffleType() != ShuffleType.ENFORCED) { - if (ConnectContext.get().getSessionVariable().isEnableBucketShuffleJoin()) { - // We need to recalculate the required property of right child, - // to make right child compatible with left child. - PhysicalProperties rightRequireProperties = calRightRequiredOfBucketShuffleJoin( - leftHashSpec, rightHashSpec); - if (!rightOutput.equals(rightRequireProperties)) { - updateChildEnforceAndCost(rightChild, rightOutput, - (DistributionSpecHash) rightRequireProperties.getDistributionSpec(), rightLowest.first); - } - childrenProperties.set(1, rightRequireProperties); - return enforceCost; + } else if ((leftHashSpec.getShuffleType() == ShuffleType.STORAGE_BUCKETED + && rightHashSpec.getShuffleType() == ShuffleType.STORAGE_BUCKETED)) { + if (bothSideShuffleKeysAreSameOrder(rightHashSpec, leftHashSpec, + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec())) { + return true; + } + if (children.get(0).getPlan() instanceof PhysicalDistribute) { + updatedForLeft = Optional.of(calAnotherSideRequired( + ShuffleType.STORAGE_BUCKETED, rightHashSpec, leftHashSpec, + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec())); + } else { + updatedForRight = Optional.of(calAnotherSideRequired( + ShuffleType.STORAGE_BUCKETED, leftHashSpec, rightHashSpec, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec())); } - updateChildEnforceAndCost(leftChild, leftOutput, - (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), leftLowest.first); - childrenProperties.set(0, requiredProperties.get(0)); } - // check right hand must distribute. - if (rightHashSpec.getShuffleType() != ShuffleType.ENFORCED) { - updateChildEnforceAndCost(rightChild, rightOutput, - (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec(), rightLowest.first); - childrenProperties.set(1, requiredProperties.get(1)); + updatedForLeft.ifPresent(physicalProperties -> updateChildEnforceAndCost(0, physicalProperties)); + updatedForRight.ifPresent(physicalProperties -> updateChildEnforceAndCost(1, physicalProperties)); + + return true; + } + + @Override + public Boolean visitPhysicalNestedLoopJoin(PhysicalNestedLoopJoin nestedLoopJoin, + Void context) { + Preconditions.checkArgument(children.size() == 2, String.format("children.size() is %d", children.size())); + Preconditions.checkArgument(childrenProperties.size() == 2); + Preconditions.checkArgument(requiredProperties.size() == 2); + DistributionSpec rightDistributionSpec = childrenProperties.get(1).getDistributionSpec(); + if (rightDistributionSpec instanceof DistributionSpecStorageGather) { + updateChildEnforceAndCost(1, PhysicalProperties.GATHER); } + return true; + } + + @Override + public Boolean visitPhysicalSetOperation(PhysicalSetOperation setOperation, Void context) { + if (children.isEmpty()) { + return true; + } + + PhysicalProperties requiredProperty = requiredProperties.get(0); + DistributionSpec requiredDistributionSpec = requiredProperty.getDistributionSpec(); + if (requiredDistributionSpec instanceof DistributionSpecGather) { + for (int i = 0; i < childrenProperties.size(); i++) { + if (childrenProperties.get(i).getDistributionSpec() instanceof DistributionSpecStorageGather) { + updateChildEnforceAndCost(i, PhysicalProperties.GATHER); + } + } + } else if (requiredDistributionSpec instanceof DistributionSpecAny) { + for (int i = 0; i < childrenProperties.size(); i++) { + if (childrenProperties.get(i).getDistributionSpec() instanceof DistributionSpecStorageAny + || childrenProperties.get(i).getDistributionSpec() instanceof DistributionSpecStorageGather + || (childrenProperties.get(i).getDistributionSpec() instanceof DistributionSpecHash + && ((DistributionSpecHash) childrenProperties.get(i).getDistributionSpec()) + .getShuffleType() == ShuffleType.NATURAL)) { + updateChildEnforceAndCost(i, PhysicalProperties.EXECUTION_ANY); + } + } + } else if (requiredDistributionSpec instanceof DistributionSpecHash) { + // TODO: should use the most common hash spec as basic + DistributionSpecHash basic = (DistributionSpecHash) requiredDistributionSpec; + for (int i = 0; i < childrenProperties.size(); i++) { + DistributionSpecHash current = (DistributionSpecHash) childrenProperties.get(i).getDistributionSpec(); + if (current.getShuffleType() != ShuffleType.EXECUTION_BUCKETED + || !bothSideShuffleKeysAreSameOrder(basic, current, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(i).getDistributionSpec())) { + PhysicalProperties target = calAnotherSideRequired( + ShuffleType.EXECUTION_BUCKETED, basic, current, + (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(), + (DistributionSpecHash) requiredProperties.get(i).getDistributionSpec()); + updateChildEnforceAndCost(i, target); + } + } + } + return true; + } - return enforceCost; + private boolean bothSideShuffleKeysAreSameOrder( + DistributionSpecHash notShuffleSideOutput, DistributionSpecHash shuffleSideOutput, + DistributionSpecHash notShuffleSideRequired, DistributionSpecHash shuffleSideRequired) { + return shuffleSideOutput.getOrderedShuffledColumns().equals( + calAnotherSideRequiredShuffleIds(notShuffleSideOutput, notShuffleSideRequired, shuffleSideRequired)); } - private PhysicalProperties calRightRequiredOfBucketShuffleJoin(DistributionSpecHash leftHashSpec, - DistributionSpecHash rightHashSpec) { - Preconditions.checkArgument(leftHashSpec.getShuffleType() != ShuffleType.ENFORCED); - DistributionSpecHash leftRequireSpec = (DistributionSpecHash) requiredProperties.get(0).getDistributionSpec(); - DistributionSpecHash rightRequireSpec = (DistributionSpecHash) requiredProperties.get(1).getDistributionSpec(); + private List calAnotherSideRequiredShuffleIds(DistributionSpecHash notShuffleSideOutput, + DistributionSpecHash notShuffleSideRequired, DistributionSpecHash shuffleSideRequired) { List rightShuffleIds = new ArrayList<>(); - for (ExprId scanId : leftHashSpec.getOrderedShuffledColumns()) { - int index = leftRequireSpec.getOrderedShuffledColumns().indexOf(scanId); + for (ExprId scanId : notShuffleSideOutput.getOrderedShuffledColumns()) { + int index = notShuffleSideRequired.getOrderedShuffledColumns().indexOf(scanId); if (index == -1) { - // when there is no exprId in leftHashSpec, we need to check EquivalenceExprIds - Set equivalentExprIds = leftHashSpec.getEquivalenceExprIdsOf(scanId); + // when there is no exprId in notShuffleSideOutput, we need to check EquivalenceExprIds + Set equivalentExprIds = notShuffleSideOutput.getEquivalenceExprIdsOf(scanId); for (ExprId alternativeExpr : equivalentExprIds) { - index = leftRequireSpec.getOrderedShuffledColumns().indexOf(alternativeExpr); + index = notShuffleSideRequired.getOrderedShuffledColumns().indexOf(alternativeExpr); if (index != -1) { break; } } } Preconditions.checkArgument(index != -1); - rightShuffleIds.add(rightRequireSpec.getOrderedShuffledColumns().get(index)); + rightShuffleIds.add(shuffleSideRequired.getOrderedShuffledColumns().get(index)); } - return new PhysicalProperties(new DistributionSpecHash(rightShuffleIds, ShuffleType.ENFORCED, - rightHashSpec.getTableId(), rightHashSpec.getSelectedIndexId(), rightHashSpec.getPartitionIds())); + return rightShuffleIds; } - private double updateChildEnforceAndCost(GroupExpression child, PhysicalProperties childOutput, - DistributionSpecHash required, Cost currentCost) { + private PhysicalProperties calAnotherSideRequired(ShuffleType shuffleType, + DistributionSpecHash notShuffleSideOutput, DistributionSpecHash shuffleSideOutput, + DistributionSpecHash notShuffleSideRequired, DistributionSpecHash shuffleSideRequired) { + List shuffleSideIds = calAnotherSideRequiredShuffleIds(notShuffleSideOutput, + notShuffleSideRequired, shuffleSideRequired); + return new PhysicalProperties(new DistributionSpecHash(shuffleSideIds, shuffleType, + shuffleSideOutput.getTableId(), shuffleSideOutput.getSelectedIndexId(), + shuffleSideOutput.getPartitionIds())); + } + + private void updateChildEnforceAndCost(int index, PhysicalProperties targetProperties) { + GroupExpression child = children.get(index); + Pair> lowest = child.getLowestCostTable().get(childrenProperties.get(index)); + PhysicalProperties output = child.getOutputProperties(childrenProperties.get(index)); + DistributionSpec target = targetProperties.getDistributionSpec(); + updateChildEnforceAndCost(child, output, target, lowest.first); + childrenProperties.set(index, targetProperties); + } + + // TODO: why add enforcer according to target and target is from requiredProperties not regular + private void updateChildEnforceAndCost(GroupExpression child, PhysicalProperties childOutput, + DistributionSpec target, Cost currentCost) { if (child.getPlan() instanceof PhysicalDistribute) { //To avoid continuous distribute operator, we just enforce the child's child childOutput = child.getInputPropertiesList(childOutput).get(0); - Pair newChildAndCost - = child.getOwnerGroup().getLowestCostPlan(childOutput).get(); + Pair newChildAndCost = child.getOwnerGroup().getLowestCostPlan(childOutput).get(); child = newChildAndCost.second; currentCost = newChildAndCost.first; } - DistributionSpec outputDistributionSpec; - outputDistributionSpec = required.withShuffleType(ShuffleType.ENFORCED); - - PhysicalProperties newOutputProperty = new PhysicalProperties(outputDistributionSpec); - GroupExpression enforcer = outputDistributionSpec.addEnforcer(child.getOwnerGroup()); + PhysicalProperties newOutputProperty = new PhysicalProperties(target); + GroupExpression enforcer = target.addEnforcer(child.getOwnerGroup()); jobContext.getCascadesContext().getMemo().addEnforcerPlan(enforcer, child.getOwnerGroup()); Cost totalCost = CostCalculator.addChildCost(enforcer.getPlan(), CostCalculator.calculateCost(enforcer, Lists.newArrayList(childOutput)), @@ -207,6 +351,5 @@ private double updateChildEnforceAndCost(GroupExpression child, PhysicalProperti enforcer.putOutputPropertiesMap(newOutputProperty, newOutputProperty); } child.getOwnerGroup().setBestPlan(enforcer, totalCost, newOutputProperty); - return enforceCost; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecAny.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecAny.java index 0bc75ffce8d998..3aa7ca226b9c2f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecAny.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecAny.java @@ -18,7 +18,7 @@ package org.apache.doris.nereids.properties; /** - * Data can be anywhere on the segments (required only). + * Data can be in any instance */ public class DistributionSpecAny extends DistributionSpec { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecExecutionAny.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecExecutionAny.java new file mode 100644 index 00000000000000..8f4299c17946cd --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecExecutionAny.java @@ -0,0 +1,37 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.properties; + +/** + * Data can be in any instance, used in PhysicalDistribute. + * Because all candidates in group could save as DistributionSpecAny's value in LowestCostPlan map + * to distinguish DistributionSpecAny, we need a new Spec to represent must shuffle require. + */ +public class DistributionSpecExecutionAny extends DistributionSpec { + + public static final DistributionSpecExecutionAny INSTANCE = new DistributionSpecExecutionAny(); + + private DistributionSpecExecutionAny() { + super(); + } + + @Override + public boolean satisfy(DistributionSpec other) { + return other instanceof DistributionSpecAny || other instanceof DistributionSpecExecutionAny; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecGather.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecGather.java index 8bbb8fff2c0794..117dbb7222240f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecGather.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecGather.java @@ -18,7 +18,7 @@ package org.apache.doris.nereids.properties; /** - * Gather distribution which put all data into one node. + * Gather distribution which put all data into one instance. */ public class DistributionSpecGather extends DistributionSpec { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java index 6a7f899c2acfd4..d61e99688a28bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecHash.java @@ -205,22 +205,11 @@ public boolean satisfy(DistributionSpec required) { return false; } - if (requiredHash.shuffleType == ShuffleType.NATURAL && this.shuffleType != ShuffleType.NATURAL) { - // this shuffle type is not natural but require natural - return false; - } - - if (requiredHash.shuffleType == ShuffleType.AGGREGATE) { + if (requiredHash.getShuffleType() == ShuffleType.REQUIRE) { return containsSatisfy(requiredHash.getOrderedShuffledColumns()); } - - // If the required property is from join and this property is not enforced, we only need to check to contain - // And more checking is in ChildrenPropertiesRegulator - if (requiredHash.shuffleType == shuffleType.JOIN && this.shuffleType != shuffleType.ENFORCED) { - return containsSatisfy(requiredHash.getOrderedShuffledColumns()); - } - - return equalsSatisfy(requiredHash.getOrderedShuffledColumns()); + return requiredHash.getShuffleType() == this.getShuffleType() + && equalsSatisfy(requiredHash.getOrderedShuffledColumns()); } private boolean containsSatisfy(List required) { @@ -253,13 +242,14 @@ public DistributionSpecHash withShuffleType(ShuffleType shuffleType) { /** * generate a new DistributionSpec after projection. */ - public DistributionSpec project(Map projections, Set obstructions) { + public DistributionSpec project(Map projections, + Set obstructions, DistributionSpec defaultAnySpec) { List orderedShuffledColumns = Lists.newArrayList(); List> equivalenceExprIds = Lists.newArrayList(); Map exprIdToEquivalenceSet = Maps.newHashMap(); for (ExprId shuffledColumn : this.orderedShuffledColumns) { if (obstructions.contains(shuffledColumn)) { - return DistributionSpecAny.INSTANCE; + return defaultAnySpec; } orderedShuffledColumns.add(projections.getOrDefault(shuffledColumn, shuffledColumn)); } @@ -267,7 +257,7 @@ public DistributionSpec project(Map projections, Set obs Set projectionEquivalenceSet = Sets.newHashSet(); for (ExprId equivalence : equivalenceSet) { if (obstructions.contains(equivalence)) { - return DistributionSpecAny.INSTANCE; + return defaultAnySpec; } projectionEquivalenceSet.add(projections.getOrDefault(equivalence, equivalence)); } @@ -275,7 +265,7 @@ public DistributionSpec project(Map projections, Set obs } for (Map.Entry exprIdSetKV : this.exprIdToEquivalenceSet.entrySet()) { if (obstructions.contains(exprIdSetKV.getKey())) { - return DistributionSpecAny.INSTANCE; + return defaultAnySpec; } if (projections.containsKey(exprIdSetKV.getKey())) { exprIdToEquivalenceSet.put(projections.get(exprIdSetKV.getKey()), exprIdSetKV.getValue()); @@ -317,19 +307,14 @@ public String toString() { * Enums for concrete shuffle type. */ public enum ShuffleType { - // 1. The following properties are the required properties for children - // require, need to satisfy the distribution spec by aggregation way. - AGGREGATE, - // require, need to satisfy the distribution spec by join way. - JOIN, - - // 2. The following properties are the output properties from some operators - // output, for olap scan node and colocate join + // require, need to satisfy the distribution spec by contains. + REQUIRE, + // output, execution only could be done on the node with data NATURAL, - // output, for all join except colocate join - BUCKETED, - // output, all distribute enforce - ENFORCED, + // output, for shuffle by execution hash method + EXECUTION_BUCKETED, + // output, for shuffle by storage hash method + STORAGE_BUCKETED, } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecReplicated.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecReplicated.java index baae83d9eacbb5..87c38f0d6bc0ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecReplicated.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecReplicated.java @@ -18,7 +18,7 @@ package org.apache.doris.nereids.properties; /** - * Data is replicated across all segments. + * Data is replicated across all instances. * Like: broadcast join. */ public class DistributionSpecReplicated extends DistributionSpec { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecStorageAny.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecStorageAny.java new file mode 100644 index 00000000000000..b8d0dd42ebb941 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecStorageAny.java @@ -0,0 +1,37 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.properties; + +/** + * Data can be in any instance, but it restricted by physical storage nodes. + * When Plan's distribution is DistributionSpecStorageAny, + * the execution on it only could be done on the node storages its data. + */ +public class DistributionSpecStorageAny extends DistributionSpec { + + public static final DistributionSpecStorageAny INSTANCE = new DistributionSpecStorageAny(); + + private DistributionSpecStorageAny() { + super(); + } + + @Override + public boolean satisfy(DistributionSpec other) { + return other instanceof DistributionSpecAny || other instanceof DistributionSpecStorageAny; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecStorageGather.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecStorageGather.java new file mode 100644 index 00000000000000..5a5f6827f8c1af --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/DistributionSpecStorageGather.java @@ -0,0 +1,38 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.properties; + +/** + * Gather distribution which put all data into one instance and + * the execution on it only could be done on the node storages its data. + */ +public class DistributionSpecStorageGather extends DistributionSpec { + + public static final DistributionSpecStorageGather INSTANCE = new DistributionSpecStorageGather(); + + public DistributionSpecStorageGather() { + super(); + } + + @Override + public boolean satisfy(DistributionSpec other) { + return other instanceof DistributionSpecGather + || other instanceof DistributionSpecStorageGather + || other instanceof DistributionSpecAny; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/EnforceMissingPropertiesHelper.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/EnforceMissingPropertiesHelper.java index 6ea0f9f06481a7..146b165d688803 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/EnforceMissingPropertiesHelper.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/EnforceMissingPropertiesHelper.java @@ -117,7 +117,7 @@ private PhysicalProperties enforceDistribution(PhysicalProperties oldOutputPrope DistributionSpec requiredDistributionSpec = required.getDistributionSpec(); if (requiredDistributionSpec instanceof DistributionSpecHash) { DistributionSpecHash requiredDistributionSpecHash = (DistributionSpecHash) requiredDistributionSpec; - outputDistributionSpec = requiredDistributionSpecHash.withShuffleType(ShuffleType.ENFORCED); + outputDistributionSpec = requiredDistributionSpecHash.withShuffleType(ShuffleType.EXECUTION_BUCKETED); } else { outputDistributionSpec = requiredDistributionSpec; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java index 54f2e6cb9de808..28bf34797788ef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/PhysicalProperties.java @@ -34,10 +34,16 @@ public class PhysicalProperties { public static PhysicalProperties ANY = new PhysicalProperties(); + public static PhysicalProperties STORAGE_ANY = new PhysicalProperties(DistributionSpecStorageAny.INSTANCE); + + public static PhysicalProperties EXECUTION_ANY = new PhysicalProperties(DistributionSpecExecutionAny.INSTANCE); + public static PhysicalProperties REPLICATED = new PhysicalProperties(DistributionSpecReplicated.INSTANCE); public static PhysicalProperties GATHER = new PhysicalProperties(DistributionSpecGather.INSTANCE); + public static PhysicalProperties STORAGE_GATHER = new PhysicalProperties(DistributionSpecStorageGather.INSTANCE); + private final OrderSpec orderSpec; private final DistributionSpec distributionSpec; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java index 5ab11c015d8ccb..c18fcc7cfb8fdd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java @@ -23,29 +23,24 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.trees.expressions.ExprId; -import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.NamedExpression; -import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; +import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.JoinHint; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalAssertNumRows; -import org.apache.doris.nereids.trees.plans.physical.PhysicalGenerate; +import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEAnchor; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalLimit; import org.apache.doris.nereids.trees.plans.physical.PhysicalNestedLoopJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; -import org.apache.doris.nereids.trees.plans.physical.PhysicalPartitionTopN; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; -import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.JoinUtils; -import com.google.common.base.Preconditions; import com.google.common.collect.Lists; import java.util.List; -import java.util.Set; import java.util.stream.Collectors; /** @@ -80,7 +75,7 @@ public List> getRequestChildrenPropertyList(GroupExpres @Override public Void visit(Plan plan, PlanContext context) { if (plan instanceof RequirePropertiesSupplier) { - RequireProperties requireProperties = ((RequirePropertiesSupplier) plan).getRequireProperties(); + RequireProperties requireProperties = ((RequirePropertiesSupplier) plan).getRequireProperties(); List requestPhysicalProperties = requireProperties.computeRequirePhysicalProperties(plan, requestPropertyFromParent); addRequestPropertyToChildren(requestPhysicalProperties); @@ -96,75 +91,68 @@ public Void visit(Plan plan, PlanContext context) { return null; } + /* ******************************************************************************************** + * sink Node, in lexicographical order + * ******************************************************************************************** */ + @Override - public Void visitAbstractPhysicalSort(AbstractPhysicalSort sort, PlanContext context) { - if (!sort.getSortPhase().isLocal()) { - addRequestPropertyToChildren(PhysicalProperties.GATHER); - } else { - addRequestPropertyToChildren(PhysicalProperties.ANY); - } + public Void visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, PlanContext context) { + addRequestPropertyToChildren(olapTableSink.getRequirePhysicalProperties()); return null; } + /* ******************************************************************************************** + * Other Node, in lexicographical order + * ******************************************************************************************** */ + @Override - public Void visitPhysicalLimit(PhysicalLimit limit, PlanContext context) { - if (limit.isGlobal()) { - addRequestPropertyToChildren(PhysicalProperties.GATHER); - } else { - addRequestPropertyToChildren(PhysicalProperties.ANY); - } + public Void visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, PlanContext context) { + addRequestPropertyToChildren(PhysicalProperties.GATHER); return null; } @Override - public Void visitPhysicalPartitionTopN(PhysicalPartitionTopN partitionTopN, PlanContext context) { - addRequestPropertyToChildren(PhysicalProperties.ANY); + public Void visitPhysicalCTEAnchor(PhysicalCTEAnchor cteAnchor, + PlanContext context) { + addRequestPropertyToChildren(PhysicalProperties.ANY, requestPropertyFromParent); return null; } @Override public Void visitPhysicalHashJoin(PhysicalHashJoin hashJoin, PlanContext context) { JoinHint hint = hashJoin.getHint(); - switch (hint) { - case BROADCAST_RIGHT: - addBroadcastJoinRequestProperty(); - break; - case SHUFFLE_RIGHT: - addShuffleJoinRequestProperty(hashJoin); - break; - case NONE: - default: - // for shuffle join - if (JoinUtils.couldShuffle(hashJoin)) { - addShuffleJoinRequestProperty(hashJoin); - } - // for broadcast join - if (JoinUtils.couldBroadcast(hashJoin)) { - addRequestPropertyToChildren(PhysicalProperties.ANY, PhysicalProperties.REPLICATED); - } - + if (hint == JoinHint.BROADCAST_RIGHT && JoinUtils.couldBroadcast(hashJoin)) { + addBroadcastJoinRequestProperty(); + return null; + } + if (hint == JoinHint.SHUFFLE_RIGHT && JoinUtils.couldShuffle(hashJoin)) { + addShuffleJoinRequestProperty(hashJoin); + return null; + } + // for shuffle join + if (JoinUtils.couldShuffle(hashJoin)) { + addShuffleJoinRequestProperty(hashJoin); + } + // for broadcast join + if (JoinUtils.couldBroadcast(hashJoin)) { + addBroadcastJoinRequestProperty(); } return null; } - private void addBroadcastJoinRequestProperty() { - addRequestPropertyToChildren(PhysicalProperties.ANY, PhysicalProperties.REPLICATED); - } - - private void addShuffleJoinRequestProperty(PhysicalHashJoin hashJoin) { - Pair, List> onClauseUsedSlots = hashJoin.getHashConjunctsExprIds(); - // shuffle join - addRequestPropertyToChildren( - PhysicalProperties.createHash( - new DistributionSpecHash(onClauseUsedSlots.first, ShuffleType.JOIN)), - PhysicalProperties.createHash( - new DistributionSpecHash(onClauseUsedSlots.second, ShuffleType.JOIN))); + @Override + public Void visitPhysicalLimit(PhysicalLimit limit, PlanContext context) { + if (limit.isGlobal()) { + addRequestPropertyToChildren(PhysicalProperties.GATHER); + } else { + addRequestPropertyToChildren(PhysicalProperties.ANY); + } + return null; } @Override public Void visitPhysicalNestedLoopJoin( PhysicalNestedLoopJoin nestedLoopJoin, PlanContext context) { - // TODO: currently doris only use NLJ to do cross join, update this if we use NLJ to do other joins. // see canParallelize() in NestedLoopJoinNode if (nestedLoopJoin.getJoinType().isCrossJoin() || nestedLoopJoin.getJoinType().isInnerJoin() || nestedLoopJoin.getJoinType().isLeftJoin()) { @@ -176,23 +164,105 @@ public Void visitPhysicalNestedLoopJoin( } @Override - public Void visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, PlanContext context) { - addRequestPropertyToChildren(PhysicalProperties.GATHER); + public Void visitPhysicalSetOperation(PhysicalSetOperation setOperation, PlanContext context) { + // intersect and except need do distinct, so we must do distribution on it. + DistributionSpec distributionRequestFromParent = requestPropertyFromParent.getDistributionSpec(); + if (distributionRequestFromParent instanceof DistributionSpecHash) { + DistributionSpecHash distributionSpecHash = (DistributionSpecHash) distributionRequestFromParent; + addRequestPropertyToChildren(createHashRequestAccordingToParent( + setOperation, distributionSpecHash, context)); + } else { + addRequestPropertyToChildren(setOperation.children().stream() + .map(Plan::getOutputExprIds) + .map(l -> PhysicalProperties.createHash(l, ShuffleType.EXECUTION_BUCKETED)) + .collect(Collectors.toList())); + } return null; } @Override - public Void visitPhysicalGenerate(PhysicalGenerate generate, PlanContext context) { - addRequestPropertyToChildren(PhysicalProperties.ANY); + public Void visitPhysicalUnion(PhysicalUnion union, PlanContext context) { + List requiredPropertyList = + Lists.newArrayListWithCapacity(context.arity()); + if (union.getConstantExprsList().isEmpty()) { + // translate requestPropertyFromParent to other children's request. + DistributionSpec distributionRequestFromParent = requestPropertyFromParent.getDistributionSpec(); + if (distributionRequestFromParent instanceof DistributionSpecHash) { + DistributionSpecHash distributionSpecHash = (DistributionSpecHash) distributionRequestFromParent; + requiredPropertyList = createHashRequestAccordingToParent(union, distributionSpecHash, context); + } else if (distributionRequestFromParent instanceof DistributionSpecGather) { + for (int i = context.arity(); i > 0; --i) { + requiredPropertyList.add(PhysicalProperties.GATHER); + } + } else { + for (int i = context.arity(); i > 0; --i) { + requiredPropertyList.add(PhysicalProperties.ANY); + } + } + + } else { + // current be could not run const expr on appropriate node, + // so if we have constant exprs on union, the output of union always any + // then any request on children is useless. + for (int i = context.arity(); i > 0; --i) { + requiredPropertyList.add(PhysicalProperties.ANY); + } + } + addRequestPropertyToChildren(requiredPropertyList); return null; } @Override - public Void visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, PlanContext context) { - addRequestPropertyToChildren(olapTableSink.getRequirePhysicalProperties()); + public Void visitAbstractPhysicalSort(AbstractPhysicalSort sort, PlanContext context) { + if (!sort.getSortPhase().isLocal()) { + addRequestPropertyToChildren(PhysicalProperties.GATHER); + } else { + addRequestPropertyToChildren(PhysicalProperties.ANY); + } return null; } + private List createHashRequestAccordingToParent( + Plan plan, DistributionSpecHash distributionRequestFromParent, PlanContext context) { + List requiredPropertyList = + Lists.newArrayListWithCapacity(context.arity()); + int[] outputOffsets = new int[distributionRequestFromParent.getOrderedShuffledColumns().size()]; + List setOperationOutputs = plan.getOutput(); + // get the offset of bucketed columns of set operation + for (int i = 0; i < setOperationOutputs.size(); i++) { + int offset = distributionRequestFromParent.getExprIdToEquivalenceSet() + .getOrDefault(setOperationOutputs.get(i).getExprId(), -1); + if (offset > 0) { + outputOffsets[offset] = i; + } + } + // use the offset to generate children's request + for (int i = 0; i < context.arity(); i++) { + List childOutput = plan.child(i).getOutput(); + List childRequest = Lists.newArrayList(); + for (int offset : outputOffsets) { + childRequest.add(childOutput.get(offset).getExprId()); + } + requiredPropertyList.add(PhysicalProperties.createHash( + childRequest, distributionRequestFromParent.getShuffleType())); + } + return requiredPropertyList; + } + + private void addBroadcastJoinRequestProperty() { + addRequestPropertyToChildren(PhysicalProperties.ANY, PhysicalProperties.REPLICATED); + } + + private void addShuffleJoinRequestProperty(PhysicalHashJoin hashJoin) { + Pair, List> onClauseUsedSlots = hashJoin.getHashConjunctsExprIds(); + // shuffle join + addRequestPropertyToChildren( + PhysicalProperties.createHash( + new DistributionSpecHash(onClauseUsedSlots.first, ShuffleType.REQUIRE)), + PhysicalProperties.createHash( + new DistributionSpecHash(onClauseUsedSlots.second, ShuffleType.REQUIRE))); + } + /** * helper function to assemble request children physical properties * @param physicalProperties one set request properties for children @@ -204,29 +274,5 @@ private void addRequestPropertyToChildren(PhysicalProperties... physicalProperti private void addRequestPropertyToChildren(List physicalProperties) { requestPropertyToChildren.add(physicalProperties); } - - private List extractExprIdFromDistinctFunction(List outputExpression) { - Set distinctAggregateFunctions = ExpressionUtils.collect(outputExpression, expr -> - expr instanceof AggregateFunction && ((AggregateFunction) expr).isDistinct() - ); - List exprIds = Lists.newArrayList(); - for (AggregateFunction aggregateFunction : distinctAggregateFunctions) { - for (Expression expr : aggregateFunction.children()) { - Preconditions.checkState(expr instanceof SlotReference, "normalize aggregate failed to" - + " normalize aggregate function " + aggregateFunction.toSql()); - exprIds.add(((SlotReference) expr).getExprId()); - } - } - return exprIds; - } - - private void addRequestHashDistribution(List hashColumns, ShuffleType shuffleType) { - List partitionedSlots = hashColumns.stream() - .map(SlotReference.class::cast) - .map(SlotReference::getExprId) - .collect(Collectors.toList()); - addRequestPropertyToChildren( - PhysicalProperties.createHash(new DistributionSpecHash(partitionedSlots, shuffleType))); - } } 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 79d89a35f82a06..bf95f4468e78e8 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 @@ -207,6 +207,9 @@ public enum RuleType { PUSH_AGGREGATE_TO_OLAP_SCAN(RuleTypeClass.REWRITE), EXTRACT_SINGLE_TABLE_EXPRESSION_FROM_DISJUNCTION(RuleTypeClass.REWRITE), HIDE_ONE_ROW_RELATION_UNDER_UNION(RuleTypeClass.REWRITE), + PUSH_PROJECT_THROUGH_UNION(RuleTypeClass.REWRITE), + MERGE_ONE_ROW_RELATION_INTO_UNION(RuleTypeClass.REWRITE), + PUSH_PROJECT_INTO_ONE_ROW_RELATION(RuleTypeClass.REWRITE), MERGE_SET_OPERATION(RuleTypeClass.REWRITE), BUILD_AGG_FOR_UNION(RuleTypeClass.REWRITE), COUNT_DISTINCT_REWRITE(RuleTypeClass.REWRITE), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 286dbebfc1289a..8ecd7b08ae8ae6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -506,11 +506,29 @@ protected boolean condition(Rule rule, Plan plan) { && (setOperation instanceof LogicalExcept || setOperation instanceof LogicalIntersect)) { throw new AnalysisException("INTERSECT and EXCEPT does not support ALL qualified"); } - + // we need to do cast before set operation, because we maybe use these slot to do shuffle + // so, we must cast it before shuffle to get correct hash code. List> castExpressions = setOperation.collectCastExpressions(); + ImmutableList.Builder newChildren = ImmutableList.builder(); + for (int i = 0; i < castExpressions.size(); i++) { + if (castExpressions.stream().allMatch(SlotReference.class::isInstance)) { + newChildren.add(setOperation.child(i)); + } else { + List projections = castExpressions.get(i).stream() + .map(e -> { + if (e instanceof SlotReference) { + return (SlotReference) e; + } else { + return new Alias(e, e.toSql()); + } + }).collect(ImmutableList.toImmutableList()); + LogicalProject logicalProject = new LogicalProject<>(projections, + setOperation.child(i)); + newChildren.add(logicalProject); + } + } List newOutputs = setOperation.buildNewOutputs(castExpressions.get(0)); - - return setOperation.withNewOutputs(newOutputs); + return setOperation.withNewOutputs(newOutputs).withChildren(newChildren.build()); }) ), RuleType.BINDING_GENERATE_SLOT.build( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java index bbe2a52469a0a9..321dbe9f762fc4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAnalysis.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.generator.TableGeneratingFunction; @@ -32,6 +33,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalGenerate; import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; @@ -74,6 +76,12 @@ public class CheckAnalysis implements AnalysisRuleFactory { GroupingScalarFunction.class, TableGeneratingFunction.class, WindowExpression.class)) + .put(LogicalOneRowRelation.class, ImmutableSet.of( + AggregateFunction.class, + GroupingScalarFunction.class, + SlotReference.class, + TableGeneratingFunction.class, + WindowExpression.class)) .put(LogicalProject.class, ImmutableSet.of( TableGeneratingFunction.class)) .put(LogicalSort.class, ImmutableSet.of( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java index a72fa10b0d6422..cf133da31fd708 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/AggregateStrategies.java @@ -369,7 +369,7 @@ private List> onePhaseAggregateWithoutDistinct( return ImmutableList.of(gatherLocalAgg); } else { RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate hashLocalAgg = gatherLocalAgg .withRequire(requireHash) .withPartitionExpressions(logicalAgg.getGroupByExpressions()); @@ -481,7 +481,7 @@ private List> twoPhaseAggregateWithCountDistinctMult return ImmutableList.of(gatherLocalGatherDistinctAgg); } else { RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate hashLocalHashGlobalAgg = gatherLocalGatherDistinctAgg .withRequireTree(requireHash.withChildren(requireHash)) .withPartitionExpressions(logicalAgg.getGroupByExpressions()); @@ -621,7 +621,7 @@ private List> threePhaseAggregateWithCount ); RequireProperties requireDistinctHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate anyLocalHashGlobalGatherDistinctAgg = anyLocalGatherGlobalGatherAgg.withChildren(ImmutableList.of( anyLocalGatherGlobalAgg @@ -636,7 +636,7 @@ private List> threePhaseAggregateWithCount .build(); } else { RequireProperties requireGroupByHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate> anyLocalHashGlobalHashDistinctAgg = anyLocalGatherGlobalGatherAgg.withRequirePropertiesAndChild(requireGroupByHash, anyLocalGatherGlobalAgg @@ -730,7 +730,7 @@ inputToBufferParam, maybeUsingStreamAgg(connectContext, logicalAgg), return ImmutableList.of(anyLocalGatherGlobalAgg); } else { RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate anyLocalHashGlobalAgg = anyLocalGatherGlobalAgg .withRequire(requireHash) @@ -841,7 +841,7 @@ private List> twoPhaseAggregateWithDistinc if (logicalAgg.getGroupByExpressions().isEmpty()) { RequireProperties requireDistinctHash = RequireProperties.of(PhysicalProperties.createHash( - distinctArguments, ShuffleType.AGGREGATE)); + distinctArguments, ShuffleType.REQUIRE)); PhysicalHashAggregate hashLocalGatherGlobalAgg = gatherLocalGatherGlobalAgg .withChildren(ImmutableList.of(gatherLocalAgg .withRequire(requireDistinctHash) @@ -853,7 +853,7 @@ private List> twoPhaseAggregateWithDistinc .build(); } else { RequireProperties requireGroupByHash = RequireProperties.of(PhysicalProperties.createHash( - logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate> hashLocalHashGlobalAgg = gatherLocalGatherGlobalAgg .withRequirePropertiesAndChild(requireGroupByHash, gatherLocalAgg .withRequire(requireGroupByHash) @@ -990,7 +990,7 @@ private List> threePhaseAggregateWithDisti requireGather, anyLocalGatherGlobalAgg); RequireProperties requireDistinctHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getDistinctArguments(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getDistinctArguments(), ShuffleType.REQUIRE)); PhysicalHashAggregate anyLocalHashGlobalGatherDistinctAgg = anyLocalGatherGlobalGatherDistinctAgg .withChildren(ImmutableList.of(anyLocalGatherGlobalAgg @@ -1006,7 +1006,7 @@ private List> threePhaseAggregateWithDisti .build(); } else { RequireProperties requireGroupByHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate anyLocalHashGlobalHashDistinctAgg = anyLocalGatherGlobalGatherDistinctAgg .withRequirePropertiesAndChild(requireGroupByHash, anyLocalGatherGlobalAgg @@ -1069,7 +1069,7 @@ private List> onePhaseAggregateWithMultiDi return ImmutableList.of(gatherLocalAgg); } else { RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate hashLocalAgg = gatherLocalAgg .withRequire(requireHash) .withPartitionExpressions(logicalAgg.getGroupByExpressions()); @@ -1156,7 +1156,7 @@ inputToBufferParam, maybeUsingStreamAgg(connectContext, logicalAgg), if (logicalAgg.getGroupByExpressions().isEmpty()) { Collection distinctArguments = logicalAgg.getDistinctArguments(); RequireProperties requireDistinctHash = RequireProperties.of(PhysicalProperties.createHash( - distinctArguments, ShuffleType.AGGREGATE)); + distinctArguments, ShuffleType.REQUIRE)); PhysicalHashAggregate hashLocalGatherGlobalAgg = anyLocalGatherGlobalAgg .withChildren(ImmutableList.of(anyLocalAgg .withRequire(requireDistinctHash) @@ -1168,7 +1168,7 @@ inputToBufferParam, maybeUsingStreamAgg(connectContext, logicalAgg), .build(); } else { RequireProperties requireHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getGroupByExpressions(), ShuffleType.REQUIRE)); PhysicalHashAggregate anyLocalHashGlobalAgg = anyLocalGatherGlobalAgg .withRequire(requireHash) .withPartitionExpressions(logicalAgg.getGroupByExpressions()); @@ -1338,7 +1338,7 @@ private List> fourPhaseAggregateWithDistin RequireProperties requireGather = RequireProperties.of(PhysicalProperties.GATHER); RequireProperties requireDistinctHash = RequireProperties.of( - PhysicalProperties.createHash(logicalAgg.getDistinctArguments(), ShuffleType.AGGREGATE)); + PhysicalProperties.createHash(logicalAgg.getDistinctArguments(), ShuffleType.REQUIRE)); //phase 2 PhysicalHashAggregate anyLocalHashGlobalAgg = new PhysicalHashAggregate<>( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOlapScanToPhysicalOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOlapScanToPhysicalOlapScan.java index 4619e05f150c34..40e0d0fbf21257 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOlapScanToPhysicalOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOlapScanToPhysicalOlapScan.java @@ -25,9 +25,10 @@ import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.PartitionType; import org.apache.doris.nereids.properties.DistributionSpec; -import org.apache.doris.nereids.properties.DistributionSpecAny; import org.apache.doris.nereids.properties.DistributionSpecHash; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; +import org.apache.doris.nereids.properties.DistributionSpecStorageAny; +import org.apache.doris.nereids.properties.DistributionSpecStorageGather; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -35,6 +36,7 @@ import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.qe.ConnectContext; import com.google.common.collect.Lists; import com.google.common.collect.Sets; @@ -75,54 +77,53 @@ private DistributionSpec convertDistribution(LogicalOlapScan olapScan) { && !colocateTableIndex.isGroupUnstable(colocateTableIndex.getGroup(olapTable.getId())); boolean isSelectUnpartition = olapTable.getPartitionInfo().getType() == PartitionType.UNPARTITIONED || olapScan.getSelectedPartitionIds().size() == 1; - if (isBelongStableCG || isSelectUnpartition) { - if (!(distributionInfo instanceof HashDistributionInfo) - || olapScan.getSelectedIndexId() != olapScan.getTable().getBaseIndexId()) { - // TODO if a mv is selected, we ignore base table's distributionInfo for now - // need improve this to handle the case if mv's distributionInfo is the same as base table - if (olapScan.getSelectedIndexId() != olapScan.getTable().getBaseIndexId()) { - HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; - List output = olapScan.getOutput(); - List baseOutput = olapScan.getOutputByIndex(olapScan.getTable().getBaseIndexId()); - List hashColumns = Lists.newArrayList(); - for (int i = 0; i < output.size(); i++) { - for (Column column : hashDistributionInfo.getDistributionColumns()) { - if (((SlotReference) output.get(i)).getColumn().get().getNameWithoutMvPrefix() - .equals(column.getName())) { - hashColumns.add(output.get(i).getExprId()); - } + if (!olapScan.getTable().isColocateTable() && olapScan.getScanTabletNum() == 1 + && !isBelongStableCG + && !ConnectContext.get().getSessionVariable().enablePipelineEngine()) { + // TODO: find a better way to handle both tablet num == 1 and colocate table together in future + return DistributionSpecStorageGather.INSTANCE; + } else if (distributionInfo instanceof HashDistributionInfo && (isBelongStableCG || isSelectUnpartition)) { + if (olapScan.getSelectedIndexId() != olapScan.getTable().getBaseIndexId()) { + HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; + List output = olapScan.getOutput(); + List baseOutput = olapScan.getOutputByIndex(olapScan.getTable().getBaseIndexId()); + List hashColumns = Lists.newArrayList(); + for (Slot slot : output) { + for (Column column : hashDistributionInfo.getDistributionColumns()) { + if (((SlotReference) slot).getColumn().get().getNameWithoutMvPrefix() + .equals(column.getName())) { + hashColumns.add(slot.getExprId()); } } - if (hashColumns.size() != hashDistributionInfo.getDistributionColumns().size()) { - for (int i = 0; i < baseOutput.size(); i++) { - for (Column column : hashDistributionInfo.getDistributionColumns()) { - if (((SlotReference) baseOutput.get(i)).getColumn().get().equals(column)) { - hashColumns.add(baseOutput.get(i).getExprId()); - } + } + if (hashColumns.size() != hashDistributionInfo.getDistributionColumns().size()) { + for (Slot slot : baseOutput) { + for (Column column : hashDistributionInfo.getDistributionColumns()) { + if (((SlotReference) slot).getColumn().get().equals(column)) { + hashColumns.add(slot.getExprId()); } } } - return new DistributionSpecHash(hashColumns, ShuffleType.NATURAL, olapScan.getTable().getId(), - olapScan.getSelectedIndexId(), Sets.newHashSet(olapScan.getSelectedPartitionIds())); } - return DistributionSpecAny.INSTANCE; - } - HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; - List output = olapScan.getOutput(); - List hashColumns = Lists.newArrayList(); - for (int i = 0; i < output.size(); i++) { - for (Column column : hashDistributionInfo.getDistributionColumns()) { - if (((SlotReference) output.get(i)).getColumn().get().equals(column)) { - hashColumns.add(output.get(i).getExprId()); + return new DistributionSpecHash(hashColumns, ShuffleType.NATURAL, olapScan.getTable().getId(), + olapScan.getSelectedIndexId(), Sets.newHashSet(olapScan.getSelectedPartitionIds())); + } else { + HashDistributionInfo hashDistributionInfo = (HashDistributionInfo) distributionInfo; + List output = olapScan.getOutput(); + List hashColumns = Lists.newArrayList(); + for (Slot slot : output) { + for (Column column : hashDistributionInfo.getDistributionColumns()) { + if (((SlotReference) slot).getColumn().get().equals(column)) { + hashColumns.add(slot.getExprId()); + } } } + return new DistributionSpecHash(hashColumns, ShuffleType.NATURAL, olapScan.getTable().getId(), + olapScan.getSelectedIndexId(), Sets.newHashSet(olapScan.getSelectedPartitionIds())); } - // TODO: need to consider colocate and dynamic partition and partition number - return new DistributionSpecHash(hashColumns, ShuffleType.NATURAL, olapScan.getTable().getId(), - olapScan.getSelectedIndexId(), Sets.newHashSet(olapScan.getSelectedPartitionIds())); } else { // RandomDistributionInfo - return DistributionSpecAny.INSTANCE; + return DistributionSpecStorageAny.INSTANCE; } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOneRowRelationToPhysicalOneRowRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOneRowRelationToPhysicalOneRowRelation.java index 95fa819bae7923..0d007d69840f43 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOneRowRelationToPhysicalOneRowRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalOneRowRelationToPhysicalOneRowRelation.java @@ -29,7 +29,7 @@ public class LogicalOneRowRelationToPhysicalOneRowRelation extends OneImplementa public Rule build() { return logicalOneRowRelation() .then(relation -> new PhysicalOneRowRelation( - relation.getProjects(), relation.buildUnionNode(), relation.getLogicalProperties())) + relation.getProjects(), relation.getLogicalProperties())) .toRule(RuleType.LOGICAL_ONE_ROW_RELATION_TO_PHYSICAL_ONE_ROW_RELATION); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalUnionToPhysicalUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalUnionToPhysicalUnion.java index 3ae88e33d52462..1b9d8e00dafc18 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalUnionToPhysicalUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalUnionToPhysicalUnion.java @@ -29,6 +29,7 @@ public class LogicalUnionToPhysicalUnion extends OneImplementationRuleFactory { public Rule build() { return logicalUnion().then(union -> new PhysicalUnion(union.getQualifier(), + union.getConstantExprsList(), union.getLogicalProperties(), union.children()) ).toRule(RuleType.LOGICAL_UNION_TO_PHYSICAL_UNION); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindow.java index 89adcf6ef7efcf..81acbce1ac5e5d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindow.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindow.java @@ -18,7 +18,7 @@ package org.apache.doris.nereids.rules.implementation; import org.apache.doris.nereids.annotation.DependsRules; -import org.apache.doris.nereids.properties.DistributionSpecHash; +import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.properties.OrderSpec; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -145,7 +145,7 @@ private List generateKeysNeedToBeSorted(OrderKeyGroup orderKeyGroup) { if (!orderKeyGroup.orderKeys.isEmpty()) { keysNeedToBeSorted.addAll(orderKeyGroup.orderKeys.stream() - .map(orderExpression -> orderExpression.getOrderKey()) + .map(OrderExpression::getOrderKey) .collect(Collectors.toList()) ); } @@ -165,7 +165,7 @@ private PhysicalWindow createPhysicalWindow(Plan root, WindowFrameGroup wi root); if (windowFrameGroup.partitionKeys.isEmpty() && requiredOrderKeys.isEmpty()) { - return physicalWindow; + return physicalWindow.withRequirePropertiesAndChild(RequireProperties.of(PhysicalProperties.GATHER), root); } // todo: WFGs in the same OKG only need same RequiredProperties @@ -174,7 +174,7 @@ private PhysicalWindow createPhysicalWindow(Plan root, WindowFrameGroup wi properties = PhysicalProperties.GATHER.withOrderSpec(new OrderSpec(requiredOrderKeys)); } else { properties = PhysicalProperties.createHash( - windowFrameGroup.partitionKeys, DistributionSpecHash.ShuffleType.ENFORCED); + windowFrameGroup.partitionKeys, ShuffleType.REQUIRE); // requiredOrderKeys contain partitionKeys, so there is no need to check if requiredOrderKeys.isEmpty() properties = properties.withOrderSpec(new OrderSpec(requiredOrderKeys)); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java index f42046d834b928..67dc893dfdbb00 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AdjustNullable.java @@ -134,6 +134,9 @@ public Plan visitLogicalRepeat(LogicalRepeat repeat, Void contex @Override public Plan visitLogicalSetOperation(LogicalSetOperation setOperation, Void context) { setOperation = (LogicalSetOperation) super.visit(setOperation, context); + if (setOperation.children().isEmpty()) { + return setOperation; + } List inputNullable = setOperation.child(0).getOutput().stream() .map(ExpressionTrait::nullable).collect(Collectors.toList()); for (int i = 1; i < setOperation.arity(); i++) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/HideOneRowRelationUnderUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/HideOneRowRelationUnderUnion.java deleted file mode 100644 index 268f5c35b42390..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/HideOneRowRelationUnderUnion.java +++ /dev/null @@ -1,74 +0,0 @@ -// Licensed to the Apache Software Foundation (ASF) under one -// or more contributor license agreements. See the NOTICE file -// distributed with this work for additional information -// regarding copyright ownership. The ASF licenses this file -// to you under the Apache License, Version 2.0 (the -// "License"); you may not use this file except in compliance -// with the License. You may obtain a copy of the License at -// -// http://www.apache.org/licenses/LICENSE-2.0 -// -// Unless required by applicable law or agreed to in writing, -// software distributed under the License is distributed on an -// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY -// KIND, either express or implied. See the License for the -// specific language governing permissions and limitations -// under the License. - -package org.apache.doris.nereids.rules.rewrite; - -import org.apache.doris.nereids.rules.Rule; -import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.rules.analysis.AnalysisRuleFactory; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; - -import com.google.common.collect.ImmutableList; - -import java.util.List; - -/** - * 1. Include oneRowRelation in the union, hide the oneRowRelation, and reduce the generation of union nodes. - * eg: select k1, k2 from t1 union select 1, 2 union select d1, d2 from t2; - * before: - * logicalUnion() - * / \ - * logicalUnion() logicalProject - * / \ - * logicalProject logicalOneRowRelation(BuildUnionNode:true) - * eg: select k1, k2 from t1 union select 1, 2 union select d1, d2 from t2; - * - * after: - * logicalUnion() - * / \ - * logicalUnion() logicalProject - * / \ - * logicalProject logicalOneRowRelation(BuildUnionNode:false) - */ -public class HideOneRowRelationUnderUnion implements AnalysisRuleFactory { - @Override - public List buildRules() { - return ImmutableList.of( - RuleType.HIDE_ONE_ROW_RELATION_UNDER_UNION.build( - logicalUnion(logicalOneRowRelation().when(LogicalOneRowRelation::buildUnionNode), any()) - .then(union -> { - List newChildren = new ImmutableList.Builder() - .add(((LogicalOneRowRelation) union.child(0)).withBuildUnionNode(false)) - .add(union.child(1)) - .build(); - return union.withChildren(newChildren); - }) - ), - RuleType.HIDE_ONE_ROW_RELATION_UNDER_UNION.build( - logicalUnion(any(), logicalOneRowRelation().when(LogicalOneRowRelation::buildUnionNode)) - .then(union -> { - List children = new ImmutableList.Builder() - .add(union.child(0)) - .add(((LogicalOneRowRelation) union.child(1)).withBuildUnionNode(false)) - .build(); - return union.withChildren(children); - }) - ) - ); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeOneRowRelationIntoUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeOneRowRelationIntoUnion.java new file mode 100644 index 00000000000000..7b115d79d3dfd3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/MergeOneRowRelationIntoUnion.java @@ -0,0 +1,66 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.util.TypeCoercionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableList.Builder; +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * merge one row relation into union, for easy to compute physical properties + */ +public class MergeOneRowRelationIntoUnion extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalUnion().when(u -> u.children().stream() + .anyMatch(LogicalOneRowRelation.class::isInstance)).then(u -> { + List> constantExprsList = Lists.newArrayList(); + List newChildren = Lists.newArrayList(); + for (Plan child : u.children()) { + if (!(child instanceof LogicalOneRowRelation)) { + newChildren.add(child); + } else { + ImmutableList.Builder constantExprs = new Builder<>(); + List projects = ((LogicalOneRowRelation) child).getProjects(); + for (int i = 0; i < projects.size(); i++) { + NamedExpression project = projects.get(i); + DataType targetType = u.getOutput().get(i).getDataType(); + if (project.getDataType().equals(targetType)) { + constantExprs.add(project); + } else { + constantExprs.add((NamedExpression) project.withChildren( + TypeCoercionUtils.castIfNotSameType(project.child(0), targetType))); + } + } + constantExprsList.add(constantExprs.build()); + } + } + return u.withChildrenAndConstExprsList(newChildren, constantExprsList); + }).toRule(RuleType.MERGE_ONE_ROW_RELATION_INTO_UNION); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectIntoOneRowRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectIntoOneRowRelation.java new file mode 100644 index 00000000000000..04faffd9afca74 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectIntoOneRowRelation.java @@ -0,0 +1,62 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; + +import java.util.Map; + +/** + * Project(OneRowRelation) -> OneRowRelation + */ +public class PushProjectIntoOneRowRelation extends OneRewriteRuleFactory { + @Override + public Rule build() { + return logicalProject(logicalOneRowRelation()).then(p -> { + Map replaceMap = Maps.newHashMap(); + Map replaceRootMap = Maps.newHashMap(); + p.child().getProjects().forEach(ne -> { + if (ne instanceof Alias) { + replaceMap.put(ne.toSlot(), ((Alias) ne).child()); + } else { + replaceMap.put(ne, ne); + } + replaceRootMap.put(ne.toSlot(), ne); + }); + ImmutableList.Builder newProjections = ImmutableList.builder(); + for (NamedExpression old : p.getProjects()) { + if (old instanceof SlotReference) { + newProjections.add(replaceRootMap.get(old)); + } else { + newProjections.add((NamedExpression) ExpressionUtils.replace(old, replaceMap)); + } + } + return p.child().withProjects(newProjections.build()); + + }).toRule(RuleType.PUSH_PROJECT_INTO_ONE_ROW_RELATION); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectThroughUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectThroughUnion.java new file mode 100644 index 00000000000000..882d0af36eb870 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushProjectThroughUnion.java @@ -0,0 +1,82 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Maps; + +import java.util.List; +import java.util.Map; + +/** + * when we do analyze, we add project on the top of SetOperation's children when we need to cast children's output + * this rule push down the project under union to let MergeUnion could do better + * TODO: this rule maybe lead to unequal transformation if cast is not monomorphism, + * maybe we need to distinguish implicit cast and explicit cast + */ +public class PushProjectThroughUnion extends OneRewriteRuleFactory { + + @Override + public Rule build() { + return logicalProject(logicalSetOperation()) + .when(project -> project.getProjects().size() == project.child().getOutput().size() + && project.getProjects().stream().allMatch(e -> { + if (e instanceof SlotReference) { + return true; + } else { + Expression expr = ExpressionUtils.getExpressionCoveredByCast(e.child(0)); + return expr instanceof SlotReference; + } + } + )) + .then(project -> { + LogicalSetOperation union = project.child(); + ImmutableList.Builder newChildren = ImmutableList.builder(); + for (Plan child : union.children()) { + Map replaceMap = Maps.newHashMap(); + for (int i = 0; i < union.getOutput().size(); i++) { + replaceMap.put(union.getOutput().get(i), child.getOutput().get(i)); + } + List childProjections = project.getProjects().stream() + .map(e -> (NamedExpression) ExpressionUtils.replace(e, replaceMap)) + .map(e -> { + if (e instanceof Alias) { + return new Alias(((Alias) e).child(), e.getName()); + } + return e; + }) + .collect(ImmutableList.toImmutableList()); + newChildren.add(new LogicalProject<>(childProjections, child)); + } + List newOutput = (List) project.getOutput(); + return union.withNewOutputs(newOutput).withChildren(newChildren.build()); + }) + .toRule(RuleType.PUSH_PROJECT_THROUGH_UNION); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java index 38856f44ba7580..acf24961df88b5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/stats/StatsCalculator.java @@ -42,13 +42,13 @@ import org.apache.doris.nereids.trees.plans.algebra.Filter; import org.apache.doris.nereids.trees.plans.algebra.Generate; import org.apache.doris.nereids.trees.plans.algebra.Limit; -import org.apache.doris.nereids.trees.plans.algebra.OneRowRelation; import org.apache.doris.nereids.trees.plans.algebra.PartitionTopN; import org.apache.doris.nereids.trees.plans.algebra.Project; import org.apache.doris.nereids.trees.plans.algebra.Repeat; import org.apache.doris.nereids.trees.plans.algebra.Scan; import org.apache.doris.nereids.trees.plans.algebra.SetOperation; import org.apache.doris.nereids.trees.plans.algebra.TopN; +import org.apache.doris.nereids.trees.plans.algebra.Union; import org.apache.doris.nereids.trees.plans.algebra.Window; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalAssertNumRows; @@ -254,7 +254,7 @@ public Statistics visitPhysicalLimit(PhysicalLimit limit, Void c @Override public Statistics visitLogicalOneRowRelation(LogicalOneRowRelation oneRowRelation, Void context) { - return computeOneRowRelation(oneRowRelation); + return computeOneRowRelation(oneRowRelation.getProjects()); } @Override @@ -396,7 +396,7 @@ public Statistics visitPhysicalRepeat(PhysicalRepeat repeat, Voi @Override public Statistics visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, Void context) { - return computeOneRowRelation(oneRowRelation); + return computeOneRowRelation(oneRowRelation.getProjects()); } @Override @@ -760,9 +760,8 @@ private Statistics computeProject(Project project) { return new Statistics(childStats.getRowCount(), columnsStats, childStats.getWidth(), childStats.getPenalty()); } - private Statistics computeOneRowRelation(OneRowRelation oneRowRelation) { - Map columnStatsMap = oneRowRelation.getProjects() - .stream() + private Statistics computeOneRowRelation(List projects) { + Map columnStatsMap = projects.stream() .map(project -> { ColumnStatistic statistic = new ColumnStatisticBuilder().setNdv(1).build(); // TODO: compute the literal size @@ -789,13 +788,33 @@ private Statistics computeEmptyRelation(EmptyRelation emptyRelation) { } private Statistics computeUnion(SetOperation setOperation) { - List head = groupExpression.child(0).getLogicalProperties().getOutput(); - Statistics headStats = groupExpression.childStatistics(0); + // TODO: refactor this for one row relation + List head = null; + Statistics headStats = null; List> childOutputs = groupExpression.children() .stream().map(ge -> ge.getLogicalProperties().getOutput()).collect(Collectors.toList()); List childStats = groupExpression.children().stream().map(Group::getStatistics).collect(Collectors.toList()); + if (setOperation instanceof Union) { + childOutputs.addAll(((Union) setOperation).getConstantExprsList().stream() + .map(l -> l.stream().map(NamedExpression::toSlot).collect(Collectors.toList())) + .collect(Collectors.toList())); + childStats.addAll(((Union) setOperation).getConstantExprsList().stream() + .map(this::computeOneRowRelation) + .collect(Collectors.toList())); + if (!((Union) setOperation).getConstantExprsList().isEmpty()) { + head = ((Union) setOperation).getConstantExprsList().get(0).stream() + .map(NamedExpression::toSlot) + .collect(Collectors.toList()); + headStats = computeOneRowRelation(((Union) setOperation).getConstantExprsList().get(0)); + } + } + if (head == null) { + head = groupExpression.child(0).getLogicalProperties().getOutput(); + headStats = groupExpression.childStatistics(0); + } + StatisticsBuilder statisticsBuilder = new StatisticsBuilder(); List unionOutput = setOperation.getOutputs(); for (int i = 0; i < head.size(); i++) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Hdfs.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Hdfs.java index 8ea2d1841421e4..2054a9ac8b11a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Hdfs.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/Hdfs.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.TVFProperties; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -63,9 +62,4 @@ public Statistics computeStats(List slots) { public R accept(ExpressionVisitor visitor, C context) { return visitor.visitHdfs(this, context); } - - @Override - public PhysicalProperties getPhysicalProperties() { - return PhysicalProperties.ANY; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/S3.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/S3.java index 3d995c6fcafb1f..c2fc81489ae206 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/S3.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/S3.java @@ -19,7 +19,6 @@ import org.apache.doris.catalog.FunctionSignature; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.TVFProperties; import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; @@ -63,9 +62,4 @@ public Statistics computeStats(List slots) { public R accept(ExpressionVisitor visitor, C context) { return visitor.visitS3(this, context); } - - @Override - public PhysicalProperties getPhysicalProperties() { - return PhysicalProperties.ANY; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java index 108474799aa1f9..a014fcc3f85b60 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Plan.java @@ -95,6 +95,10 @@ default Set getOutputSet() { return ImmutableSet.copyOf(getOutput()); } + default List getOutputExprIds() { + return getOutput().stream().map(NamedExpression::getExprId).collect(Collectors.toList()); + } + default Set getOutputExprIdSet() { return getOutput().stream().map(NamedExpression::getExprId).collect(Collectors.toSet()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Union.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Union.java new file mode 100644 index 00000000000000..f6623102d40003 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Union.java @@ -0,0 +1,30 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.algebra; + +import org.apache.doris.nereids.trees.expressions.NamedExpression; + +import java.util.List; + +/** + * Common interface for logical/physical Union. + */ +public interface Union extends SetOperation { + + List> getConstantExprsList(); +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOneRowRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOneRowRelation.java index 6685640dff8c1e..4a1f4936247e5f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOneRowRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOneRowRelation.java @@ -43,14 +43,12 @@ public class LogicalOneRowRelation extends LogicalLeaf implements OneRowRelation, OutputPrunable { private final List projects; - private final boolean buildUnionNode; public LogicalOneRowRelation(List projects) { - this(projects, true, Optional.empty(), Optional.empty()); + this(projects, Optional.empty(), Optional.empty()); } private LogicalOneRowRelation(List projects, - boolean buildUnionNode, Optional groupExpression, Optional logicalProperties) { super(PlanType.LOGICAL_ONE_ROW_RELATION, groupExpression, logicalProperties); @@ -59,7 +57,6 @@ private LogicalOneRowRelation(List projects, Preconditions.checkArgument(projects.stream().noneMatch(p -> p.containsType(AggregateFunction.class)), "OneRowRelation can not contains any aggregate function"); this.projects = ImmutableList.copyOf(Objects.requireNonNull(projects, "projects can not be null")); - this.buildUnionNode = buildUnionNode; } @Override @@ -79,13 +76,12 @@ public List getExpressions() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new LogicalOneRowRelation(projects, buildUnionNode, - groupExpression, Optional.of(logicalPropertiesSupplier.get())); + return new LogicalOneRowRelation(projects, groupExpression, Optional.of(logicalPropertiesSupplier.get())); } @Override public Plan withLogicalProperties(Optional logicalProperties) { - return new LogicalOneRowRelation(projects, buildUnionNode, Optional.empty(), logicalProperties); + return new LogicalOneRowRelation(projects, Optional.empty(), logicalProperties); } @Override @@ -98,8 +94,7 @@ public List computeOutput() { @Override public String toString() { return Utils.toSqlString("LogicalOneRowRelation", - "projects", projects, - "buildUnionNode", buildUnionNode + "projects", projects ); } @@ -115,25 +110,16 @@ public boolean equals(Object o) { return false; } LogicalOneRowRelation that = (LogicalOneRowRelation) o; - return Objects.equals(projects, that.projects) - && Objects.equals(buildUnionNode, that.buildUnionNode); + return Objects.equals(projects, that.projects); } @Override public int hashCode() { - return Objects.hash(projects, buildUnionNode); - } - - public boolean buildUnionNode() { - return buildUnionNode; + return Objects.hash(projects); } public LogicalOneRowRelation withProjects(List namedExpressions) { - return new LogicalOneRowRelation(namedExpressions, buildUnionNode, Optional.empty(), Optional.empty()); - } - - public Plan withBuildUnionNode(boolean buildUnionNode) { - return new LogicalOneRowRelation(projects, buildUnionNode, Optional.empty(), Optional.empty()); + return new LogicalOneRowRelation(namedExpressions, Optional.empty(), Optional.empty()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnion.java index 934df533f723ee..b5feeb82c5c09c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalUnion.java @@ -19,12 +19,16 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Union; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; +import com.google.common.collect.ImmutableList; + import java.util.List; import java.util.Objects; import java.util.Optional; @@ -32,7 +36,10 @@ /** * Logical Union. */ -public class LogicalUnion extends LogicalSetOperation implements OutputPrunable { +public class LogicalUnion extends LogicalSetOperation implements Union, OutputPrunable { + + // in doris, we use union node to present one row relation + private final List> constantExprsList; // When there is an agg on the union and there is a filter on the agg, // it is necessary to keep the filter on the agg and push the filter down to each child of the union. private final boolean hasPushedFilter; @@ -40,25 +47,47 @@ public class LogicalUnion extends LogicalSetOperation implements OutputPrunable public LogicalUnion(Qualifier qualifier, List inputs) { super(PlanType.LOGICAL_UNION, qualifier, inputs); this.hasPushedFilter = false; + this.constantExprsList = ImmutableList.of(); } - public LogicalUnion(Qualifier qualifier, List outputs, boolean hasPushedFilter, - List inputs) { + public LogicalUnion(Qualifier qualifier, List outputs, + List> constantExprsList, boolean hasPushedFilter, List inputs) { super(PlanType.LOGICAL_UNION, qualifier, outputs, inputs); this.hasPushedFilter = hasPushedFilter; + this.constantExprsList = ImmutableList.copyOf( + Objects.requireNonNull(constantExprsList, "constantExprsList should not be null")); } - public LogicalUnion(Qualifier qualifier, List outputs, boolean hasPushedFilter, + public LogicalUnion(Qualifier qualifier, List outputs, + List> constantExprsList, boolean hasPushedFilter, Optional groupExpression, Optional logicalProperties, List inputs) { super(PlanType.LOGICAL_UNION, qualifier, outputs, groupExpression, logicalProperties, inputs); this.hasPushedFilter = hasPushedFilter; + this.constantExprsList = ImmutableList.copyOf( + Objects.requireNonNull(constantExprsList, "constantExprsList should not be null")); + } + + public boolean hasPushedFilter() { + return hasPushedFilter; + } + + public List> getConstantExprsList() { + return constantExprsList; + } + + @Override + public List getExpressions() { + return constantExprsList.stream().flatMap(List::stream).collect(ImmutableList.toImmutableList()); } @Override public String toString() { - return Utils.toSqlString("LogicalUnion", "qualifier", qualifier, "outputs", outputs, "hasPushedFilter", - hasPushedFilter); + return Utils.toSqlString("LogicalUnion", + "qualifier", qualifier, + "outputs", outputs, + "constantExprsList", constantExprsList, + "hasPushedFilter", hasPushedFilter); } @Override @@ -70,12 +99,13 @@ public boolean equals(Object o) { return false; } LogicalUnion that = (LogicalUnion) o; - return super.equals(that) && hasPushedFilter == that.hasPushedFilter; + return super.equals(that) && hasPushedFilter == that.hasPushedFilter + && Objects.equals(constantExprsList, that.constantExprsList); } @Override public int hashCode() { - return Objects.hash(super.hashCode(), hasPushedFilter); + return Objects.hash(super.hashCode(), hasPushedFilter, constantExprsList); } @Override @@ -85,35 +115,40 @@ public R accept(PlanVisitor visitor, C context) { @Override public LogicalUnion withChildren(List children) { - return new LogicalUnion(qualifier, outputs, hasPushedFilter, children); + return new LogicalUnion(qualifier, outputs, constantExprsList, hasPushedFilter, children); } @Override public LogicalUnion withGroupExpression(Optional groupExpression) { - return new LogicalUnion(qualifier, outputs, hasPushedFilter, groupExpression, + return new LogicalUnion(qualifier, outputs, constantExprsList, hasPushedFilter, groupExpression, Optional.of(getLogicalProperties()), children); } @Override public LogicalUnion withLogicalProperties(Optional logicalProperties) { - return new LogicalUnion(qualifier, outputs, hasPushedFilter, Optional.empty(), logicalProperties, children); + return new LogicalUnion(qualifier, outputs, constantExprsList, hasPushedFilter, + Optional.empty(), logicalProperties, children); } @Override public LogicalUnion withNewOutputs(List newOutputs) { - return new LogicalUnion(qualifier, newOutputs, hasPushedFilter, Optional.empty(), Optional.empty(), children); + return new LogicalUnion(qualifier, newOutputs, constantExprsList, + hasPushedFilter, Optional.empty(), Optional.empty(), children); } - public LogicalUnion withAllQualifier() { - return new LogicalUnion(Qualifier.ALL, outputs, hasPushedFilter, Optional.empty(), Optional.empty(), children); + public LogicalUnion withChildrenAndConstExprsList( + List children, List> constantExprsList) { + return new LogicalUnion(qualifier, outputs, constantExprsList, hasPushedFilter, children); } - public boolean hasPushedFilter() { - return hasPushedFilter; + public LogicalUnion withAllQualifier() { + return new LogicalUnion(Qualifier.ALL, outputs, constantExprsList, hasPushedFilter, + Optional.empty(), Optional.empty(), children); } public LogicalUnion withHasPushedFilter() { - return new LogicalUnion(qualifier, outputs, true, Optional.empty(), Optional.empty(), children); + return new LogicalUnion(qualifier, outputs, constantExprsList, true, + Optional.empty(), Optional.empty(), children); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOneRowRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOneRowRelation.java index b780057bbdfefb..4d2f7069c24501 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOneRowRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOneRowRelation.java @@ -42,22 +42,18 @@ public class PhysicalOneRowRelation extends PhysicalLeaf implements OneRowRelation { private final List projects; - private final boolean buildUnionNode; - public PhysicalOneRowRelation(List projects, boolean buildUnionNode, - LogicalProperties logicalProperties) { - this(projects, buildUnionNode, Optional.empty(), logicalProperties, null, null); + public PhysicalOneRowRelation(List projects, LogicalProperties logicalProperties) { + this(projects, Optional.empty(), logicalProperties, null, null); } private PhysicalOneRowRelation(List projects, - boolean buildUnionNode, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { super(PlanType.PHYSICAL_ONE_ROW_RELATION, groupExpression, logicalProperties, physicalProperties, statistics); this.projects = ImmutableList.copyOf(Objects.requireNonNull(projects, "projects can not be null")); - this.buildUnionNode = buildUnionNode; } @Override @@ -77,21 +73,20 @@ public List getExpressions() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new PhysicalOneRowRelation(projects, buildUnionNode, groupExpression, + return new PhysicalOneRowRelation(projects, groupExpression, logicalPropertiesSupplier.get(), physicalProperties, statistics); } @Override public Plan withLogicalProperties(Optional logicalProperties) { - return new PhysicalOneRowRelation(projects, buildUnionNode, Optional.empty(), + return new PhysicalOneRowRelation(projects, Optional.empty(), logicalProperties.get(), physicalProperties, statistics); } @Override public String toString() { return Utils.toSqlString("PhysicalOneRowRelation[" + id.asInt() + "]" + getGroupIdAsString(), - "expressions", projects, - "buildUnionNode", buildUnionNode + "expressions", projects ); } @@ -104,23 +99,18 @@ public boolean equals(Object o) { return false; } PhysicalOneRowRelation that = (PhysicalOneRowRelation) o; - return Objects.equals(projects, that.projects) - && buildUnionNode == that.buildUnionNode; + return Objects.equals(projects, that.projects); } @Override public int hashCode() { - return Objects.hash(projects, buildUnionNode); + return Objects.hash(projects); } @Override public PhysicalOneRowRelation withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalOneRowRelation(projects, buildUnionNode, groupExpression, + return new PhysicalOneRowRelation(projects, groupExpression, logicalPropertiesSupplier.get(), physicalProperties, statistics); } - - public boolean notBuildUnionNode() { - return !buildUnionNode; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalUnion.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalUnion.java index 025397ac920db2..ff95c848251b5b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalUnion.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalUnion.java @@ -20,38 +20,59 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Union; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; +import com.google.common.collect.ImmutableList; + import java.util.List; +import java.util.Objects; import java.util.Optional; /** * Physical Union. */ -public class PhysicalUnion extends PhysicalSetOperation { +public class PhysicalUnion extends PhysicalSetOperation implements Union { + + // in doris, we use union node to present one row relation + private final List> constantExprsList; public PhysicalUnion(Qualifier qualifier, - LogicalProperties logicalProperties, - List inputs) { + List> constantExprsList, + LogicalProperties logicalProperties, + List inputs) { super(PlanType.PHYSICAL_UNION, qualifier, logicalProperties, inputs); + this.constantExprsList = ImmutableList.copyOf( + Objects.requireNonNull(constantExprsList, "constantExprsList should not be null")); } public PhysicalUnion(Qualifier qualifier, - Optional groupExpression, - LogicalProperties logicalProperties, - List inputs) { + List> constantExprsList, + Optional groupExpression, + LogicalProperties logicalProperties, + List inputs) { super(PlanType.PHYSICAL_UNION, qualifier, groupExpression, logicalProperties, inputs); + this.constantExprsList = ImmutableList.copyOf( + Objects.requireNonNull(constantExprsList, "constantExprsList should not be null")); } - public PhysicalUnion(Qualifier qualifier, + public PhysicalUnion(Qualifier qualifier, List> constantExprsList, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics, List inputs) { super(PlanType.PHYSICAL_UNION, qualifier, groupExpression, logicalProperties, physicalProperties, statistics, inputs); + this.constantExprsList = ImmutableList.copyOf( + Objects.requireNonNull(constantExprsList, "constantExprsList should not be null")); + + } + + public List> getConstantExprsList() { + return constantExprsList; } @Override @@ -63,30 +84,31 @@ public R accept(PlanVisitor visitor, C context) { public String toString() { return Utils.toSqlString("PhysicalUnion" + getGroupIdAsString(), "qualifier", qualifier, + "constantExprsList", constantExprsList, "stats", statistics); } @Override public PhysicalUnion withChildren(List children) { - return new PhysicalUnion(qualifier, getLogicalProperties(), children); + return new PhysicalUnion(qualifier, constantExprsList, getLogicalProperties(), children); } @Override public PhysicalUnion withGroupExpression( Optional groupExpression) { - return new PhysicalUnion(qualifier, groupExpression, getLogicalProperties(), children); + return new PhysicalUnion(qualifier, constantExprsList, groupExpression, getLogicalProperties(), children); } @Override public PhysicalUnion withLogicalProperties( Optional logicalProperties) { - return new PhysicalUnion(qualifier, Optional.empty(), logicalProperties.get(), children); + return new PhysicalUnion(qualifier, constantExprsList, Optional.empty(), logicalProperties.get(), children); } @Override public PhysicalUnion withPhysicalPropertiesAndStats( PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalUnion(qualifier, Optional.empty(), + return new PhysicalUnion(qualifier, constantExprsList, Optional.empty(), getLogicalProperties(), physicalProperties, statistics, children); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java index 9e31f7467e1739..bfe1594425e8a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/JoinUtils.java @@ -194,13 +194,27 @@ public static Expression swapEqualToForChildrenOrder(EqualTo equalTo, Set } } + /** + * return true if we should do bucket shuffle join when translate plan. + */ + public static boolean shouldBucketShuffleJoin(AbstractPhysicalJoin join) { + DistributionSpec rightDistributionSpec = join.right().getPhysicalProperties().getDistributionSpec(); + if (!(rightDistributionSpec instanceof DistributionSpecHash)) { + return false; + } + DistributionSpecHash rightHash = (DistributionSpecHash) rightDistributionSpec; + return rightHash.getShuffleType() == ShuffleType.STORAGE_BUCKETED; + } + /** * return true if we should do broadcast join when translate plan. */ public static boolean shouldBroadcastJoin(AbstractPhysicalJoin join) { PhysicalPlan right = join.right(); - DistributionSpec rightDistributionSpec = right.getPhysicalProperties().getDistributionSpec(); - return rightDistributionSpec instanceof DistributionSpecReplicated; + if (right instanceof PhysicalDistribute) { + return ((PhysicalDistribute) right).getDistributionSpec() instanceof DistributionSpecReplicated; + } + return false; } /** @@ -211,6 +225,7 @@ public static boolean shouldColocateJoin(AbstractPhysicalJoin join) { - if (ConnectContext.get() == null - || !ConnectContext.get().getSessionVariable().isEnableBucketShuffleJoin()) { - return false; - } - DistributionSpec joinDistributionSpec = join.getPhysicalProperties().getDistributionSpec(); - DistributionSpec leftDistributionSpec = join.left().getPhysicalProperties().getDistributionSpec(); - DistributionSpec rightDistributionSpec = join.right().getPhysicalProperties().getDistributionSpec(); - if (join.left() instanceof PhysicalDistribute) { - return false; - } - if (!(joinDistributionSpec instanceof DistributionSpecHash) - || !(leftDistributionSpec instanceof DistributionSpecHash) - || !(rightDistributionSpec instanceof DistributionSpecHash)) { - return false; - } - DistributionSpecHash leftHash = (DistributionSpecHash) leftDistributionSpec; - // when we plan a bucket shuffle join, the left should not add a distribution enforce. - // so its shuffle type should be NATURAL(olap scan node or result of colocate join / bucket shuffle join with - // left child's shuffle type is also NATURAL), or be BUCKETED(result of join / agg). - if (leftHash.getShuffleType() != ShuffleType.BUCKETED && leftHash.getShuffleType() != ShuffleType.NATURAL) { - return false; - } - // there must use left as required and join as source. - // Because after property derive upper node's properties is contains lower node - // if their properties are satisfy. - return joinDistributionSpec.satisfy(leftDistributionSpec); - } - /** * could do colocate join with left and right child distribution spec. */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java index 1e80ce17e89aa9..5da04b1f813aef 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/EsScanNode.java @@ -72,20 +72,20 @@ public class EsScanNode extends ExternalScanNode { private static final Logger LOG = LogManager.getLogger(EsScanNode.class); - private EsTablePartitions esTablePartitions; - private EsTable table; + private final EsTablePartitions esTablePartitions; + private final EsTable table; private QueryBuilder queryBuilder; private boolean isFinalized = false; - public EsScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName) { - this(id, desc, planNodeName, false); + public EsScanNode(PlanNodeId id, TupleDescriptor desc) { + this(id, desc, false); } /** * For multicatalog es. **/ - public EsScanNode(PlanNodeId id, TupleDescriptor desc, String planNodeName, boolean esExternalTable) { - super(id, desc, planNodeName, StatisticalType.ES_SCAN_NODE, false); + public EsScanNode(PlanNodeId id, TupleDescriptor desc, boolean esExternalTable) { + super(id, desc, "EsScanNode", StatisticalType.ES_SCAN_NODE, false); if (esExternalTable) { EsExternalTable externalTable = (EsExternalTable) (desc.getTable()); table = externalTable.getEsTable(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java index 879238f1ca54d9..6fa4955a319d24 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/OlapScanNode.java @@ -1542,6 +1542,8 @@ public String getSelectedIndexName() { public void finalizeForNereids() { computeNumNodes(); computeStatsForNereids(); + // distributionColumnIds is used for one backend node agg optimization, nereids do not support it. + distributionColumnIds.clear(); } private void computeStatsForNereids() { @@ -1564,18 +1566,9 @@ Set getDistributionColumnNames() { public void updateRequiredSlots(PlanTranslatorContext context, Set requiredByProjectSlotIdSet) { outputColumnUniqueIds.clear(); - distributionColumnIds.clear(); - - Set distColumnName = getDistributionColumnNames(); - - int columnId = 0; for (SlotDescriptor slot : context.getTupleDesc(this.getTupleId()).getSlots()) { if (requiredByProjectSlotIdSet.contains(slot.getId()) && slot.getColumn() != null) { outputColumnUniqueIds.add(slot.getColumn().getUniqueId()); - if (distColumnName.contains(slot.getColumn().getName().toLowerCase())) { - distributionColumnIds.add(columnId); - } - columnId++; } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java index 2a386c15c85141..b9def52802cd18 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/SingleNodePlanner.java @@ -1976,7 +1976,7 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef, SelectStmt s case BROKER: throw new RuntimeException("Broker external table is not supported, try to use table function please"); case ELASTICSEARCH: - scanNode = new EsScanNode(ctx.getNextNodeId(), tblRef.getDesc(), "EsScanNode"); + scanNode = new EsScanNode(ctx.getNextNodeId(), tblRef.getDesc()); break; case HIVE: throw new RuntimeException("Hive external table is not supported, try to use hive catalog please"); @@ -2016,7 +2016,7 @@ private PlanNode createScanNode(Analyzer analyzer, TableRef tblRef, SelectStmt s StatisticalType.MAX_COMPUTE_SCAN_NODE, true); break; case ES_EXTERNAL_TABLE: - scanNode = new EsScanNode(ctx.getNextNodeId(), tblRef.getDesc(), "EsScanNode", true); + scanNode = new EsScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); break; case JDBC_EXTERNAL_TABLE: scanNode = new JdbcScanNode(ctx.getNextNodeId(), tblRef.getDesc(), true); diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java index 23bb0d4ba5e236..a14f25ca960b31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/Coordinator.java @@ -2328,7 +2328,8 @@ public BucketShuffleJoinController(Map> fragmentIdT // check whether the node fragment is bucket shuffle join fragment private boolean isBucketShuffleJoin(int fragmentId, PlanNode node) { if (ConnectContext.get() != null) { - if (!ConnectContext.get().getSessionVariable().isEnableBucketShuffleJoin()) { + if (!ConnectContext.get().getSessionVariable().isEnableBucketShuffleJoin() + && !ConnectContext.get().getSessionVariable().isEnableNereidsPlanner()) { return false; } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java index 55c2921bb4ef47..631a89b7762402 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java @@ -121,7 +121,7 @@ public void testHintWithReorderCrossJoin() throws Exception { DistributionSpec spec = dis.getDistributionSpec(); Assertions.assertTrue(spec instanceof DistributionSpecHash); DistributionSpecHash hashSpec = (DistributionSpecHash) spec; - Assertions.assertEquals(ShuffleType.ENFORCED, + Assertions.assertEquals(ShuffleType.EXECUTION_BUCKETED, hashSpec.getShuffleType()); return true; }), physicalDistribute()), diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java index 5ac1e8cc2f1fc0..2f6a973503a2bc 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriverTest.java @@ -219,7 +219,7 @@ Pair, List> getOnClauseUsedSlots( leftMap.put(new ExprId(1), 0); PhysicalProperties left = new PhysicalProperties(new DistributionSpecHash( Lists.newArrayList(new ExprId(0)), - ShuffleType.ENFORCED, + ShuffleType.EXECUTION_BUCKETED, 0, Sets.newHashSet(0L), ImmutableList.of(Sets.newHashSet(new ExprId(0), new ExprId(1))), @@ -228,7 +228,7 @@ Pair, List> getOnClauseUsedSlots( PhysicalProperties right = new PhysicalProperties(new DistributionSpecHash( Lists.newArrayList(new ExprId(2)), - ShuffleType.ENFORCED, + ShuffleType.EXECUTION_BUCKETED, 1, Sets.newHashSet(1L) )); @@ -240,7 +240,7 @@ Pair, List> getOnClauseUsedSlots( Assertions.assertTrue(result.getOrderSpec().getOrderKeys().isEmpty()); Assertions.assertTrue(result.getDistributionSpec() instanceof DistributionSpecHash); DistributionSpecHash actual = (DistributionSpecHash) result.getDistributionSpec(); - Assertions.assertEquals(ShuffleType.BUCKETED, actual.getShuffleType()); + Assertions.assertEquals(ShuffleType.EXECUTION_BUCKETED, actual.getShuffleType()); // check merged Assertions.assertEquals(3, actual.getExprIdToEquivalenceSet().size()); } @@ -308,12 +308,12 @@ public void testGlobalPhaseAggregate() { new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.createHash(ImmutableList.of(partition), ShuffleType.AGGREGATE)), + RequireProperties.of(PhysicalProperties.createHash(ImmutableList.of(partition), ShuffleType.REQUIRE)), groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); DistributionSpecHash childHash = new DistributionSpecHash(Lists.newArrayList(partition.getExprId()), - ShuffleType.BUCKETED); + ShuffleType.EXECUTION_BUCKETED); PhysicalProperties child = new PhysicalProperties(childHash, new OrderSpec(Lists.newArrayList( new OrderKey(new SlotReference("ignored", IntegerType.INSTANCE), true, true)))); @@ -323,7 +323,7 @@ public void testGlobalPhaseAggregate() { Assertions.assertTrue(result.getOrderSpec().getOrderKeys().isEmpty()); Assertions.assertTrue(result.getDistributionSpec() instanceof DistributionSpecHash); DistributionSpecHash actual = (DistributionSpecHash) result.getDistributionSpec(); - Assertions.assertEquals(ShuffleType.BUCKETED, actual.getShuffleType()); + Assertions.assertEquals(ShuffleType.EXECUTION_BUCKETED, actual.getShuffleType()); Assertions.assertEquals(Lists.newArrayList(partition).stream() .map(SlotReference::getExprId).collect(Collectors.toList()), actual.getOrderedShuffledColumns()); @@ -355,7 +355,7 @@ public void testAggregateWithoutGroupBy() { public void testLocalQuickSort() { SlotReference key = new SlotReference("col1", IntegerType.INSTANCE); List orderKeys = Lists.newArrayList(new OrderKey(key, true, true)); - PhysicalQuickSort sort = new PhysicalQuickSort(orderKeys, SortPhase.LOCAL_SORT, logicalProperties, groupPlan); + PhysicalQuickSort sort = new PhysicalQuickSort<>(orderKeys, SortPhase.LOCAL_SORT, logicalProperties, groupPlan); GroupExpression groupExpression = new GroupExpression(sort); PhysicalProperties child = new PhysicalProperties(DistributionSpecReplicated.INSTANCE, new OrderSpec(Lists.newArrayList( @@ -417,7 +417,7 @@ public void testLimit() { List orderKeys = Lists.newArrayList(new OrderKey(key, true, true)); PhysicalLimit limit = new PhysicalLimit<>(10, 10, LimitPhase.ORIGIN, logicalProperties, groupPlan); GroupExpression groupExpression = new GroupExpression(limit); - PhysicalProperties child = new PhysicalProperties(DistributionSpecReplicated.INSTANCE, + PhysicalProperties child = new PhysicalProperties(DistributionSpecGather.INSTANCE, new OrderSpec(orderKeys)); ChildOutputPropertyDeriver deriver = new ChildOutputPropertyDeriver(Lists.newArrayList(child)); @@ -434,7 +434,7 @@ public void testAssertNumRows() { groupPlan ); GroupExpression groupExpression = new GroupExpression(assertNumRows); - PhysicalProperties child = new PhysicalProperties(DistributionSpecReplicated.INSTANCE, new OrderSpec()); + PhysicalProperties child = new PhysicalProperties(DistributionSpecGather.INSTANCE, new OrderSpec()); ChildOutputPropertyDeriver deriver = new ChildOutputPropertyDeriver(Lists.newArrayList(child)); PhysicalProperties result = deriver.getOutputProperties(groupExpression); Assertions.assertEquals(PhysicalProperties.GATHER, result); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/DistributionSpecHashTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/DistributionSpecHashTest.java index 49b3837dad3fc1..bb38f668015e95 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/DistributionSpecHashTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/DistributionSpecHashTest.java @@ -47,18 +47,18 @@ public void testMerge() { naturalMap ); - Map joinMap = Maps.newHashMap(); - joinMap.put(new ExprId(1), 0); - joinMap.put(new ExprId(4), 0); - joinMap.put(new ExprId(3), 1); - joinMap.put(new ExprId(5), 1); + Map requireMap = Maps.newHashMap(); + requireMap.put(new ExprId(1), 0); + requireMap.put(new ExprId(4), 0); + requireMap.put(new ExprId(3), 1); + requireMap.put(new ExprId(5), 1); DistributionSpecHash join = new DistributionSpecHash( Lists.newArrayList(new ExprId(1), new ExprId(5)), - ShuffleType.JOIN, + ShuffleType.REQUIRE, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(1), new ExprId(4)), Sets.newHashSet(new ExprId(3), new ExprId(5))), - joinMap + requireMap ); Map expectedMap = Maps.newHashMap(); @@ -101,7 +101,7 @@ public void testProject() { projects.put(new ExprId(2), new ExprId(5)); Set obstructions = Sets.newHashSet(); - DistributionSpec after = original.project(projects, obstructions); + DistributionSpec after = original.project(projects, obstructions, DistributionSpecAny.INSTANCE); Assertions.assertTrue(after instanceof DistributionSpecHash); DistributionSpecHash afterHash = (DistributionSpecHash) after; Assertions.assertEquals(Lists.newArrayList(new ExprId(0), new ExprId(5)), afterHash.getOrderedShuffledColumns()); @@ -119,41 +119,41 @@ public void testProject() { // have obstructions obstructions.add(new ExprId(3)); - after = original.project(projects, obstructions); + after = original.project(projects, obstructions, DistributionSpecAny.INSTANCE); Assertions.assertTrue(after instanceof DistributionSpecAny); } @Test public void testSatisfyAny() { DistributionSpec required = DistributionSpecAny.INSTANCE; - DistributionSpecHash join = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.JOIN); - DistributionSpecHash aggregate = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.AGGREGATE); - DistributionSpecHash enforce = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.ENFORCED); + DistributionSpecHash require = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.REQUIRE); + DistributionSpecHash storageBucketed = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.STORAGE_BUCKETED); + DistributionSpecHash executionBucketed = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.EXECUTION_BUCKETED); DistributionSpecHash natural = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.NATURAL); - Assertions.assertTrue(join.satisfy(required)); - Assertions.assertTrue(aggregate.satisfy(required)); - Assertions.assertTrue(enforce.satisfy(required)); + Assertions.assertTrue(require.satisfy(required)); + Assertions.assertTrue(storageBucketed.satisfy(required)); + Assertions.assertTrue(executionBucketed.satisfy(required)); Assertions.assertTrue(natural.satisfy(required)); } @Test public void testNotSatisfyOther() { - DistributionSpecHash join = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.JOIN); - DistributionSpecHash aggregate = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.AGGREGATE); - DistributionSpecHash enforce = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.ENFORCED); + DistributionSpecHash require = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.REQUIRE); + DistributionSpecHash storageBucketed = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.STORAGE_BUCKETED); + DistributionSpecHash executionBucketed = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.EXECUTION_BUCKETED); DistributionSpecHash natural = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.NATURAL); DistributionSpec gather = DistributionSpecGather.INSTANCE; - Assertions.assertFalse(join.satisfy(gather)); - Assertions.assertFalse(aggregate.satisfy(gather)); - Assertions.assertFalse(enforce.satisfy(gather)); + Assertions.assertFalse(require.satisfy(gather)); + Assertions.assertFalse(storageBucketed.satisfy(gather)); + Assertions.assertFalse(executionBucketed.satisfy(gather)); Assertions.assertFalse(natural.satisfy(gather)); DistributionSpec replicated = DistributionSpecReplicated.INSTANCE; - Assertions.assertFalse(join.satisfy(replicated)); - Assertions.assertFalse(aggregate.satisfy(replicated)); - Assertions.assertFalse(enforce.satisfy(replicated)); + Assertions.assertFalse(require.satisfy(replicated)); + Assertions.assertFalse(storageBucketed.satisfy(replicated)); + Assertions.assertFalse(executionBucketed.satisfy(replicated)); Assertions.assertFalse(natural.satisfy(replicated)); } @@ -197,27 +197,27 @@ public void testSatisfyNaturalHash() { natural3Map ); - DistributionSpecHash join = new DistributionSpecHash( + DistributionSpecHash require = new DistributionSpecHash( Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.JOIN, + ShuffleType.REQUIRE, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), natural2Map ); - DistributionSpecHash enforce = new DistributionSpecHash( + DistributionSpecHash storageBucketed = new DistributionSpecHash( Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.ENFORCED, + ShuffleType.STORAGE_BUCKETED, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), natural2Map ); - DistributionSpecHash aggregate = new DistributionSpecHash( + DistributionSpecHash executionBucketed = new DistributionSpecHash( Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.AGGREGATE, + ShuffleType.EXECUTION_BUCKETED, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), @@ -231,144 +231,49 @@ public void testSatisfyNaturalHash() { // require slots is not contained by target Assertions.assertFalse(natural2.satisfy(natural1)); // other shuffle type with same order - Assertions.assertFalse(join.satisfy(natural2)); - Assertions.assertFalse(aggregate.satisfy(natural2)); - Assertions.assertFalse(enforce.satisfy(natural2)); + Assertions.assertFalse(require.satisfy(natural2)); + Assertions.assertFalse(executionBucketed.satisfy(natural2)); + Assertions.assertFalse(storageBucketed.satisfy(natural2)); } @Test - public void testSatisfyJoinHash() { - Map join1Map = Maps.newHashMap(); - join1Map.put(new ExprId(0), 0); - join1Map.put(new ExprId(1), 0); - join1Map.put(new ExprId(2), 1); - join1Map.put(new ExprId(3), 1); - DistributionSpecHash join1 = new DistributionSpecHash( + public void testSatisfyRequiredHash() { + Map require1Map = Maps.newHashMap(); + require1Map.put(new ExprId(0), 0); + require1Map.put(new ExprId(1), 0); + require1Map.put(new ExprId(2), 1); + require1Map.put(new ExprId(3), 1); + DistributionSpecHash require1 = new DistributionSpecHash( Lists.newArrayList(new ExprId(0), new ExprId(2)), - ShuffleType.JOIN, + ShuffleType.REQUIRE, 0, Sets.newHashSet(0L), Lists.newArrayList(Sets.newHashSet(new ExprId(0), new ExprId(1)), Sets.newHashSet(new ExprId(2), new ExprId(3))), - join1Map + require1Map ); - Map join2Map = Maps.newHashMap(); - join2Map.put(new ExprId(1), 0); - join2Map.put(new ExprId(2), 1); - DistributionSpecHash join2 = new DistributionSpecHash( + Map require2Map = Maps.newHashMap(); + require2Map.put(new ExprId(1), 0); + require2Map.put(new ExprId(2), 1); + DistributionSpecHash require2 = new DistributionSpecHash( Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.JOIN, + ShuffleType.REQUIRE, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - join2Map + require2Map ); - Map join3Map = Maps.newHashMap(); - join3Map.put(new ExprId(2), 0); - join3Map.put(new ExprId(1), 1); - DistributionSpecHash join3 = new DistributionSpecHash( + Map require3Map = Maps.newHashMap(); + require3Map.put(new ExprId(2), 0); + require3Map.put(new ExprId(1), 1); + DistributionSpecHash require3 = new DistributionSpecHash( Lists.newArrayList(new ExprId(2), new ExprId(1)), - ShuffleType.JOIN, + ShuffleType.REQUIRE, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(2)), Sets.newHashSet(new ExprId(1))), - join3Map - ); - - DistributionSpecHash natural = new DistributionSpecHash( - Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.NATURAL, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - join2Map - ); - - DistributionSpecHash enforce = new DistributionSpecHash( - Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.ENFORCED, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - join2Map - ); - - DistributionSpecHash aggregate = new DistributionSpecHash( - Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.AGGREGATE, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - join2Map - ); - - // require is same order - Assertions.assertTrue(join1.satisfy(join2)); - // require contains all sets but order is not same - Assertions.assertTrue(join1.satisfy(join3)); - // require slots is not contained by target - Assertions.assertFalse(join3.satisfy(join1)); - // other shuffle type with same order - Assertions.assertTrue(natural.satisfy(join2)); - Assertions.assertTrue(aggregate.satisfy(join2)); - Assertions.assertTrue(enforce.satisfy(join2)); - // other shuffle type contain all set but order is not same - Assertions.assertTrue(natural.satisfy(join3)); - Assertions.assertTrue(aggregate.satisfy(join3)); - Assertions.assertFalse(enforce.satisfy(join3)); - } - - @Test - public void testSatisfyAggregateHash() { - Map aggregate1Map = Maps.newHashMap(); - aggregate1Map.put(new ExprId(0), 0); - aggregate1Map.put(new ExprId(1), 0); - aggregate1Map.put(new ExprId(2), 1); - aggregate1Map.put(new ExprId(3), 1); - DistributionSpecHash aggregate1 = new DistributionSpecHash( - Lists.newArrayList(new ExprId(0), new ExprId(2)), - ShuffleType.AGGREGATE, - 0, - Sets.newHashSet(0L), - Lists.newArrayList(Sets.newHashSet(new ExprId(0), new ExprId(1)), Sets.newHashSet(new ExprId(2), new ExprId(3))), - aggregate1Map - ); - - Map aggregate2Map = Maps.newHashMap(); - aggregate2Map.put(new ExprId(1), 0); - aggregate2Map.put(new ExprId(2), 1); - DistributionSpecHash aggregate2 = new DistributionSpecHash( - Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.AGGREGATE, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - aggregate2Map - ); - - Map aggregate3Map = Maps.newHashMap(); - aggregate3Map.put(new ExprId(2), 0); - aggregate3Map.put(new ExprId(1), 1); - DistributionSpecHash aggregate3 = new DistributionSpecHash( - Lists.newArrayList(new ExprId(2), new ExprId(1)), - ShuffleType.AGGREGATE, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(2)), Sets.newHashSet(new ExprId(1))), - aggregate3Map - ); - - Map aggregate4Map = Maps.newHashMap(); - aggregate4Map.put(new ExprId(2), 0); - aggregate4Map.put(new ExprId(3), 1); - DistributionSpecHash aggregate4 = new DistributionSpecHash( - Lists.newArrayList(new ExprId(2), new ExprId(3)), - ShuffleType.AGGREGATE, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(2)), Sets.newHashSet(new ExprId(3))), - aggregate4Map + require3Map ); DistributionSpecHash natural = new DistributionSpecHash( @@ -377,152 +282,71 @@ public void testSatisfyAggregateHash() { 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - aggregate2Map + require2Map ); - DistributionSpecHash enforce = new DistributionSpecHash( + DistributionSpecHash storageBucketed = new DistributionSpecHash( Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.ENFORCED, + ShuffleType.STORAGE_BUCKETED, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - aggregate2Map + require2Map ); - DistributionSpecHash join = new DistributionSpecHash( + DistributionSpecHash executionBucketed = new DistributionSpecHash( Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.JOIN, + ShuffleType.EXECUTION_BUCKETED, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - aggregate2Map + require2Map ); // require is same order - Assertions.assertTrue(aggregate1.satisfy(aggregate2)); + Assertions.assertTrue(require1.satisfy(require2)); // require contains all sets but order is not same - Assertions.assertTrue(aggregate1.satisfy(aggregate3)); - // require do not contain all set but has the same number slot - Assertions.assertFalse(aggregate1.satisfy(aggregate4)); + Assertions.assertTrue(require1.satisfy(require3)); // require slots is not contained by target - Assertions.assertFalse(aggregate3.satisfy(aggregate1)); + Assertions.assertFalse(require3.satisfy(require1)); // other shuffle type with same order - Assertions.assertTrue(natural.satisfy(aggregate2)); - Assertions.assertTrue(join.satisfy(aggregate2)); - Assertions.assertTrue(enforce.satisfy(aggregate2)); + Assertions.assertTrue(natural.satisfy(require2)); + Assertions.assertTrue(executionBucketed.satisfy(require2)); + Assertions.assertTrue(storageBucketed.satisfy(require2)); // other shuffle type contain all set but order is not same - Assertions.assertTrue(natural.satisfy(aggregate3)); - Assertions.assertTrue(join.satisfy(aggregate3)); - Assertions.assertTrue(enforce.satisfy(aggregate3)); - } - - @Test - public void testSatisfyEnforceHash() { - Map enforce1Map = Maps.newHashMap(); - enforce1Map.put(new ExprId(0), 0); - enforce1Map.put(new ExprId(1), 0); - enforce1Map.put(new ExprId(2), 1); - enforce1Map.put(new ExprId(3), 1); - DistributionSpecHash enforce1 = new DistributionSpecHash( - Lists.newArrayList(new ExprId(0), new ExprId(2)), - ShuffleType.ENFORCED, - 0, - Sets.newHashSet(0L), - Lists.newArrayList(Sets.newHashSet(new ExprId(0), new ExprId(1)), Sets.newHashSet(new ExprId(2), new ExprId(3))), - enforce1Map - ); - - Map enforce2Map = Maps.newHashMap(); - enforce2Map.put(new ExprId(1), 0); - enforce2Map.put(new ExprId(2), 1); - DistributionSpecHash enforce2 = new DistributionSpecHash( - Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.ENFORCED, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - enforce2Map - ); - - Map enforce3Map = Maps.newHashMap(); - enforce3Map.put(new ExprId(2), 0); - enforce3Map.put(new ExprId(1), 1); - DistributionSpecHash enforce3 = new DistributionSpecHash( - Lists.newArrayList(new ExprId(2), new ExprId(1)), - ShuffleType.ENFORCED, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(2)), Sets.newHashSet(new ExprId(1))), - enforce3Map - ); - - DistributionSpecHash join = new DistributionSpecHash( - Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.JOIN, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - enforce2Map - ); - - DistributionSpecHash natural = new DistributionSpecHash( - Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.NATURAL, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - enforce2Map - ); - - DistributionSpecHash aggregate = new DistributionSpecHash( - Lists.newArrayList(new ExprId(1), new ExprId(2)), - ShuffleType.AGGREGATE, - 1, - Sets.newHashSet(1L), - Lists.newArrayList(Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - enforce2Map - ); - - // require is same order - Assertions.assertTrue(enforce1.satisfy(enforce2)); - // require contains all sets but order is not same - Assertions.assertFalse(enforce1.satisfy(enforce3)); - // require slots is not contained by target - Assertions.assertFalse(enforce2.satisfy(enforce1)); - // other shuffle type with same order - Assertions.assertTrue(join.satisfy(enforce2)); - Assertions.assertTrue(aggregate.satisfy(enforce2)); - Assertions.assertTrue(natural.satisfy(enforce2)); + Assertions.assertTrue(natural.satisfy(require3)); + Assertions.assertTrue(executionBucketed.satisfy(require3)); + Assertions.assertTrue(storageBucketed.satisfy(require3)); } @Test public void testHashEqualSatisfyWithDifferentLength() { - Map enforce1Map = Maps.newHashMap(); - enforce1Map.put(new ExprId(0), 0); - enforce1Map.put(new ExprId(1), 1); - enforce1Map.put(new ExprId(2), 2); - DistributionSpecHash enforce1 = new DistributionSpecHash( + Map bucketed1Map = Maps.newHashMap(); + bucketed1Map.put(new ExprId(0), 0); + bucketed1Map.put(new ExprId(1), 1); + bucketed1Map.put(new ExprId(2), 2); + DistributionSpecHash bucketed1 = new DistributionSpecHash( Lists.newArrayList(new ExprId(0), new ExprId(1), new ExprId(2)), - ShuffleType.ENFORCED, + ShuffleType.EXECUTION_BUCKETED, 0, Sets.newHashSet(0L), Lists.newArrayList(Sets.newHashSet(new ExprId(0)), Sets.newHashSet(new ExprId(1)), Sets.newHashSet(new ExprId(2))), - enforce1Map + bucketed1Map ); - Map enforce2Map = Maps.newHashMap(); - enforce2Map.put(new ExprId(0), 0); - enforce2Map.put(new ExprId(1), 1); - DistributionSpecHash enforce2 = new DistributionSpecHash( + Map bucketed2Map = Maps.newHashMap(); + bucketed2Map.put(new ExprId(0), 0); + bucketed2Map.put(new ExprId(1), 1); + DistributionSpecHash bucketed2 = new DistributionSpecHash( Lists.newArrayList(new ExprId(0), new ExprId(1)), - ShuffleType.ENFORCED, + ShuffleType.EXECUTION_BUCKETED, 1, Sets.newHashSet(1L), Lists.newArrayList(Sets.newHashSet(new ExprId(0)), Sets.newHashSet(new ExprId(1))), - enforce2Map + bucketed2Map ); - Assertions.assertFalse(enforce1.satisfy(enforce2)); - Assertions.assertFalse(enforce2.satisfy(enforce1)); + Assertions.assertFalse(bucketed1.satisfy(bucketed2)); + Assertions.assertFalse(bucketed2.satisfy(bucketed1)); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/DistributionSpecTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/DistributionSpecTest.java index 0a09c8a63af32f..688e99e4fa981f 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/DistributionSpecTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/DistributionSpecTest.java @@ -30,7 +30,7 @@ public void testSatisfy() { DistributionSpec replicated = DistributionSpecReplicated.INSTANCE; DistributionSpec any = DistributionSpecAny.INSTANCE; DistributionSpec gather = DistributionSpecGather.INSTANCE; - DistributionSpec hash = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.JOIN); + DistributionSpec hash = new DistributionSpecHash(Lists.newArrayList(), ShuffleType.REQUIRE); Assertions.assertTrue(replicated.satisfy(any)); Assertions.assertTrue(gather.satisfy(any)); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java index 0c9044eb3df190..1df7067c2b088d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/properties/RequestPropertyDeriverTest.java @@ -110,8 +110,8 @@ Pair, List> getHashConjunctsExprIds() { List> expected = Lists.newArrayList(); expected.add(Lists.newArrayList( - new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(0)), ShuffleType.JOIN)), - new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(1)), ShuffleType.JOIN)) + new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(0)), ShuffleType.REQUIRE)), + new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(1)), ShuffleType.REQUIRE)) )); Assertions.assertEquals(expected, actual); } @@ -136,8 +136,8 @@ Pair, List> getHashConjunctsExprIds() { List> expected = Lists.newArrayList(); expected.add(Lists.newArrayList( - new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(0)), ShuffleType.JOIN)), - new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(1)), ShuffleType.JOIN)) + new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(0)), ShuffleType.REQUIRE)), + new PhysicalProperties(new DistributionSpecHash(Lists.newArrayList(new ExprId(1)), ShuffleType.REQUIRE)) )); expected.add(Lists.newArrayList(PhysicalProperties.ANY, PhysicalProperties.REPLICATED)); Assertions.assertEquals(expected, actual); @@ -174,7 +174,7 @@ public void testGlobalAggregate() { new AggregateParam(AggPhase.GLOBAL, AggMode.BUFFER_TO_RESULT), true, logicalProperties, - RequireProperties.of(PhysicalProperties.createHash(ImmutableList.of(partition), ShuffleType.AGGREGATE)), + RequireProperties.of(PhysicalProperties.createHash(ImmutableList.of(partition), ShuffleType.REQUIRE)), groupPlan ); GroupExpression groupExpression = new GroupExpression(aggregate); @@ -184,7 +184,7 @@ public void testGlobalAggregate() { List> expected = Lists.newArrayList(); expected.add(Lists.newArrayList(PhysicalProperties.createHash(new DistributionSpecHash( Lists.newArrayList(partition.getExprId()), - ShuffleType.AGGREGATE + ShuffleType.REQUIRE )))); Assertions.assertEquals(expected, actual); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/planner/DistributedPlannerTest.java b/fe/fe-core/src/test/java/org/apache/doris/planner/DistributedPlannerTest.java index 2551a9d06a7438..45aab8366fde1c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/planner/DistributedPlannerTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/planner/DistributedPlannerTest.java @@ -52,6 +52,7 @@ public class DistributedPlannerTest { public static void setUp() throws Exception { UtFrameUtils.createDorisCluster(runningDir); ctx = UtFrameUtils.createDefaultCtx(); + ctx.getSessionVariable().setEnableNereidsPlanner(false); String createDbStmtStr = "create database db1;"; CreateDbStmt createDbStmt = (CreateDbStmt) UtFrameUtils.parseAndAnalyzeStmt(createDbStmtStr, ctx); Env.getCurrentEnv().createDb(createDbStmt); 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 a25aced3030e69..42fdc0d0631374 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,45 +2,44 @@ -- !ds_shape_11 -- CteAnchor[cteId= ( CTEId#4=] ) --CteProducer[cteId= ( CTEId#4=] ) -----PhysicalProject -------PhysicalUnion ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------PhysicalProject -------------------------filter((('s' = 'w') OR ('s' = 's'))) ---------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))(('s' = 'w') OR ('s' = 's'))) -----------------------------PhysicalOlapScan[date_dim] +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk) +------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------PhysicalProject +----------------------filter((('s' = 'w') OR ('s' = 's'))) +------------------------PhysicalOlapScan[store_sales] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((('s' = 'w') OR ('s' = 's'))) ---------------------------PhysicalOlapScan[customer] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk) ---------------------PhysicalDistribute -----------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------filter((('w' = 'w') OR ('w' = 's'))) -----------------------------PhysicalOlapScan[web_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))(('w' = 'w') OR ('w' = 's'))) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute +------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))(('s' = 'w') OR ('s' = 's'))) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((('s' = 'w') OR ('s' = 's'))) +------------------------PhysicalOlapScan[customer] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ----------------------PhysicalProject ------------------------filter((('w' = 'w') OR ('w' = 's'))) ---------------------------PhysicalOlapScan[customer] +--------------------------PhysicalOlapScan[web_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(((date_dim.d_year = 2001) OR (date_dim.d_year = 2002))(('w' = 'w') OR ('w' = 's'))) +----------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((('w' = 'w') OR ('w' = 's'))) +------------------------PhysicalOlapScan[customer] --PhysicalTopN ----PhysicalDistribute ------PhysicalTopN diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out index eb27fab212c869..5750cb0006f408 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out @@ -20,7 +20,8 @@ PhysicalTopN ------------------------------PhysicalProject --------------------------------filter((date_dim.d_date <= 1998-05-06)(date_dim.d_date >= 1998-04-06)) ----------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------filter(i_category IN ('Books', 'Sports', 'Men')) -----------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter(i_category IN ('Books', 'Sports', 'Men')) +------------------------------PhysicalOlapScan[item] 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 c0e488a09a9c96..d655442bd043d8 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 @@ -5,75 +5,79 @@ CteAnchor[cteId= ( CTEId#8=] ) ----PhysicalProject ------hashJoin[INNER_JOIN](item.i_brand_id = t.brand_id)(item.i_class_id = t.class_id)(item.i_category_id = t.category_id) --------PhysicalIntersect +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = iss.i_item_sk) +----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk) +------------------PhysicalProject +--------------------PhysicalOlapScan[store_sales] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((d1.d_year <= 2002)(d1.d_year >= 2000)) +------------------------PhysicalOlapScan[date_dim] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = ics.i_item_sk) +----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d2.d_date_sk) +------------------PhysicalProject +--------------------PhysicalOlapScan[catalog_sales] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((d2.d_year >= 2000)(d2.d_year <= 2002)) +------------------------PhysicalOlapScan[date_dim] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = iws.i_item_sk) +----------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = d3.d_date_sk) +------------------PhysicalProject +--------------------PhysicalOlapScan[web_sales] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((d3.d_year <= 2002)(d3.d_year >= 2000)) +------------------------PhysicalOlapScan[date_dim] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +--------PhysicalDistribute ----------PhysicalProject -------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = iss.i_item_sk) ---------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk) +------------PhysicalOlapScan[item] +--CteAnchor[cteId= ( CTEId#10=] ) +----CteProducer[cteId= ( CTEId#10=] ) +------hashAgg[LOCAL] +--------PhysicalUnion +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------PhysicalProject ------------------PhysicalOlapScan[store_sales] ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((d1.d_year <= 2002)(d1.d_year >= 2000)) +--------------------filter((date_dim.d_year <= 2002)(date_dim.d_year >= 2000)) ----------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalOlapScan[item] -----------PhysicalProject -------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = ics.i_item_sk) ---------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d2.d_date_sk) +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------PhysicalProject ------------------PhysicalOlapScan[catalog_sales] ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((d2.d_year >= 2000)(d2.d_year <= 2002)) +--------------------filter((date_dim.d_year <= 2002)(date_dim.d_year >= 2000)) ----------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalOlapScan[item] -----------PhysicalProject -------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = iws.i_item_sk) ---------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = d3.d_date_sk) +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ----------------PhysicalProject ------------------PhysicalOlapScan[web_sales] ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((d3.d_year <= 2002)(d3.d_year >= 2000)) +--------------------filter((date_dim.d_year >= 2000)(date_dim.d_year <= 2002)) ----------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalOlapScan[item] ---------PhysicalDistribute -----------PhysicalProject -------------PhysicalOlapScan[item] ---CteAnchor[cteId= ( CTEId#10=] ) -----CteProducer[cteId= ( CTEId#10=] ) -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[store_sales] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_year <= 2002)(date_dim.d_year >= 2000)) -------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[catalog_sales] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_year <= 2002)(date_dim.d_year >= 2000)) -------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[web_sales] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_year >= 2000)(date_dim.d_year <= 2002)) -------------------------PhysicalOlapScan[date_dim] ----PhysicalTopN ------PhysicalDistribute --------PhysicalTopN @@ -102,8 +106,9 @@ CteAnchor[cteId= ( CTEId#8=] ) ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) --------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalAssertNumRows ------------------------------PhysicalDistribute @@ -127,8 +132,9 @@ CteAnchor[cteId= ( CTEId#8=] ) ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) --------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalAssertNumRows ------------------------------PhysicalDistribute @@ -152,8 +158,9 @@ CteAnchor[cteId= ( CTEId#8=] ) ----------------------------------------------PhysicalProject ------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) --------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[item] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalAssertNumRows ------------------------------PhysicalDistribute 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 62483c4de0f2dd..e2953ab328518f 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 @@ -33,9 +33,10 @@ PhysicalTopN ----------------------------------------------PhysicalProject ------------------------------------------------filter(c_birth_month IN (1, 2, 4, 7, 8, 10)) --------------------------------------------------PhysicalOlapScan[customer] ---------------------------------------------PhysicalProject -----------------------------------------------filter(ca_state IN ('WA', 'GA', 'NC', 'ME', 'WY', 'OK', 'IN')) -------------------------------------------------PhysicalOlapScan[customer_address] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter(ca_state IN ('WA', 'GA', 'NC', 'ME', 'WY', 'OK', 'IN')) +--------------------------------------------------PhysicalOlapScan[customer_address] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter((cast(cd_gender as VARCHAR(*)) = 'F')(cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out index 329e91f73ef9e6..e970e12f97ea3c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out @@ -7,10 +7,11 @@ CteAnchor[cteId= ( CTEId#4=] ) --------hashAgg[LOCAL] ----------PhysicalProject ------------hashJoin[INNER_JOIN](date_dim.d_date_sk = wscs.sold_date_sk) ---------------PhysicalProject -----------------PhysicalUnion +--------------PhysicalUnion +----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalOlapScan[web_sales] +----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalOlapScan[catalog_sales] --------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out index b322aab515ac89..9a85f030a23990 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out @@ -27,6 +27,7 @@ PhysicalTopN ----------------------PhysicalProject ------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) --------------------------PhysicalOlapScan[promotion] -----------------PhysicalProject -------------------PhysicalOlapScan[item] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out index f1fec2b391f179..e773eae3049644 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 @@ -26,8 +26,9 @@ PhysicalTopN --------------------------------PhysicalProject ----------------------------------filter((date_dim.d_year = 1999)) ------------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------filter(s_state IN ('MO', 'AL', 'MI', 'TN', 'LA', 'SC')) 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 f6803d5601a285..ed1c8ba9cb8012 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 @@ -16,8 +16,9 @@ CteAnchor[cteId= ( CTEId#2=] ) ----------------------PhysicalProject ------------------------filter((date_dim.d_year = 2002)) --------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------PhysicalOlapScan[customer_address] +--------------PhysicalDistribute +----------------PhysicalProject +------------------PhysicalOlapScan[customer_address] --PhysicalTopN ----PhysicalDistribute ------PhysicalTopN diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out index 912280a2e89b1d..e835f410d8654a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out @@ -36,8 +36,9 @@ CteAnchor[cteId= ( CTEId#6=] ) --------------------------PhysicalProject ----------------------------filter((ws.d_year = 2000)d_qoy IN (1, 2, 3)) ------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalOlapScan[customer_address] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer_address] ----PhysicalQuickSort ------PhysicalDistribute --------PhysicalQuickSort diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out index 3bee39ba9a75e7..fe5d9b14c078e5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out @@ -1,94 +1,99 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_33 -- PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] +--PhysicalTopN +----hashAgg[LOCAL] +------PhysicalUnion --------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) -----------------------------------PhysicalDistribute +----------PhysicalProject +------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) ---------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((customer_address.ca_gmt_offset = -5.00)) +------------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((item.i_category = 'Home')) +--------------------PhysicalOlapScan[item] +--------PhysicalDistribute +----------PhysicalProject +------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter((item.i_category = 'Home')) -------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) -----------------------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) ---------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((customer_address.ca_gmt_offset = -5.00)) +------------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((item.i_category = 'Home')) +--------------------PhysicalOlapScan[item] +--------PhysicalDistribute +----------PhysicalProject +------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] --------------------PhysicalProject -----------------------filter((item.i_category = 'Home')) -------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] +----------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) -----------------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) ---------------------------------------PhysicalOlapScan[customer_address] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((item.i_category = 'Home')) -------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((customer_address.ca_gmt_offset = -5.00)) +------------------------------------PhysicalOlapScan[customer_address] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((item.i_category = 'Home')) +--------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out index 4a747defc7f135..93dac990e56444 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 @@ -31,8 +31,9 @@ PhysicalTopN --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject ------------------------------------------PhysicalOlapScan[customer] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer_address] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[customer_demographics] 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 5db6313ba256d1..c4b812b0b08d36 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,65 +2,64 @@ -- !ds_shape_4 -- CteAnchor[cteId= ( CTEId#6=] ) --CteProducer[cteId= ( CTEId#6=] ) -----PhysicalProject -------PhysicalUnion ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk) +----PhysicalUnion +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_customer_sk = store_sales.ss_customer_sk) +----------------------PhysicalProject +------------------------filter(((('s' = 'c') OR ('s' = 's')) OR ('s' = 'w'))) +--------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------filter(((('s' = 'c') OR ('s' = 's')) OR ('s' = 'w'))) -----------------------------PhysicalOlapScan[store_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(((('s' = 'c') OR ('s' = 's')) OR ('s' = 'w'))) -------------------------------PhysicalOlapScan[customer] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(((date_dim.d_year = 1999) OR (date_dim.d_year = 2000))((('s' = 'c') OR ('s' = 's')) OR ('s' = 'w'))) ---------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](customer.c_customer_sk = catalog_sales.cs_bill_customer_sk) +----------------------filter(((date_dim.d_year = 1999) OR (date_dim.d_year = 2000))((('s' = 'c') OR ('s' = 's')) OR ('s' = 'w'))) +------------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_customer_sk = catalog_sales.cs_bill_customer_sk) +----------------------PhysicalProject +------------------------filter(((('c' = 'c') OR ('c' = 's')) OR ('c' = 'w'))) +--------------------------PhysicalOlapScan[catalog_sales] +----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------filter(((('c' = 'c') OR ('c' = 's')) OR ('c' = 'w'))) -----------------------------PhysicalOlapScan[catalog_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(((('c' = 'c') OR ('c' = 's')) OR ('c' = 'w'))) -------------------------------PhysicalOlapScan[customer] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(((date_dim.d_year = 1999) OR (date_dim.d_year = 2000))((('c' = 'c') OR ('c' = 's')) OR ('c' = 'w'))) ---------------------------PhysicalOlapScan[date_dim] ---------PhysicalProject -----------hashAgg[GLOBAL] -------------PhysicalDistribute ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk) +----------------------filter(((date_dim.d_year = 1999) OR (date_dim.d_year = 2000))((('c' = 'c') OR ('c' = 's')) OR ('c' = 'w'))) +------------------------PhysicalOlapScan[date_dim] +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_customer_sk = web_sales.ws_bill_customer_sk) +----------------------PhysicalProject +------------------------filter(((('w' = 'c') OR ('w' = 's')) OR ('w' = 'w'))) +--------------------------PhysicalOlapScan[web_sales] +----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------filter(((('w' = 'c') OR ('w' = 's')) OR ('w' = 'w'))) -----------------------------PhysicalOlapScan[web_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(((('w' = 'c') OR ('w' = 's')) OR ('w' = 'w'))) -------------------------------PhysicalOlapScan[customer] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(((date_dim.d_year = 1999) OR (date_dim.d_year = 2000))((('w' = 'c') OR ('w' = 's')) OR ('w' = 'w'))) ---------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter(((date_dim.d_year = 1999) OR (date_dim.d_year = 2000))((('w' = 'c') OR ('w' = 's')) OR ('w' = 'w'))) +------------------------PhysicalOlapScan[date_dim] --PhysicalTopN ----PhysicalDistribute ------PhysicalTopN 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 4a3c96c502bb5f..e84dbdaabe0fb7 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 @@ -22,16 +22,18 @@ PhysicalTopN ----------------------------------PhysicalProject ------------------------------------filter((date_dim.d_qoy = 2)(date_dim.d_year = 2000)) --------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter(i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out index f86ecfbb990db4..f1c91e1f35673a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out @@ -29,8 +29,9 @@ PhysicalTopN ----------------------------PhysicalProject ------------------------------filter(s_city IN ('Five Points', 'Centerville', 'Oak Grove', 'Fairview', 'Liberty')) --------------------------------PhysicalOlapScan[store] ---------------------PhysicalProject -----------------------PhysicalOlapScan[customer_address] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer_address] ----------PhysicalDistribute ------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk) --------------PhysicalProject diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out index 5d2dff0b768e7e..5af21c50d8ea11 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out @@ -5,31 +5,30 @@ CteAnchor[cteId= ( CTEId#0=] ) ----PhysicalProject ------PhysicalWindow --------PhysicalQuickSort -----------PhysicalDistribute -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1)))) -------------------------------------------PhysicalOlapScan[date_dim] +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +--------------------------------------filter((((date_dim.d_year = 2001) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2002) AND (date_dim.d_moy = 1)))) +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] --PhysicalProject ----PhysicalTopN ------PhysicalDistribute 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 557f4a5e33de68..f8d87cba93c406 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,91 +1,82 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -------------------PhysicalWindow ---------------------PhysicalQuickSort -----------------------PhysicalDistribute -------------------------PhysicalQuickSort ---------------------------PhysicalWindow -----------------------------PhysicalQuickSort -------------------------------PhysicalDistribute ---------------------------------PhysicalQuickSort -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] +--PhysicalTopN +----hashAgg[LOCAL] +------PhysicalUnion +--------PhysicalProject +----------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalWindow +------------------PhysicalQuickSort +--------------------PhysicalDistribute +----------------------PhysicalQuickSort +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](ws.ws_order_number = wr.wr_order_number)(item = wr.wr_item_sk) +------------------------------------PhysicalProject +--------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------PhysicalOlapScan[web_returns] +------------------------------------hashJoin[INNER_JOIN](ws.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------filter((ws.ws_net_paid > 0.00)(ws.ws_quantity > 0)(ws.ws_net_profit > 1.00)) +------------------------------------------PhysicalOlapScan[web_sales] --------------------------------------PhysicalDistribute -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](ws.ws_order_number = wr.wr_order_number)(item = wr.wr_item_sk) -----------------------------------------------PhysicalProject -------------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------------hashJoin[INNER_JOIN](ws.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------------------------PhysicalProject ---------------------------------------------------filter((ws.ws_net_paid > 0.00)(ws.ws_quantity > 0)(ws.ws_net_profit > 1.00)) -----------------------------------------------------PhysicalOlapScan[web_sales] -------------------------------------------------PhysicalDistribute ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -------------------PhysicalWindow ---------------------PhysicalQuickSort -----------------------PhysicalDistribute -------------------------PhysicalQuickSort ---------------------------PhysicalWindow -----------------------------PhysicalQuickSort -------------------------------PhysicalDistribute ---------------------------------PhysicalQuickSort -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------PhysicalOlapScan[date_dim] +--------PhysicalProject +----------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalWindow +------------------PhysicalQuickSort +--------------------PhysicalDistribute +----------------------PhysicalQuickSort +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](cs.cs_order_number = cr.cr_order_number)(item = cr.cr_item_sk) +------------------------------------PhysicalProject +--------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------PhysicalOlapScan[catalog_returns] +------------------------------------hashJoin[INNER_JOIN](cs.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------filter((cs.cs_net_paid > 0.00)(cs.cs_quantity > 0)(cs.cs_net_profit > 1.00)) +------------------------------------------PhysicalOlapScan[catalog_sales] --------------------------------------PhysicalDistribute -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](cs.cs_order_number = cr.cr_order_number)(item = cr.cr_item_sk) -----------------------------------------------PhysicalProject -------------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------------hashJoin[INNER_JOIN](cs.cs_sold_date_sk = date_dim.d_date_sk) -------------------------------------------------PhysicalProject ---------------------------------------------------filter((cs.cs_net_paid > 0.00)(cs.cs_quantity > 0)(cs.cs_net_profit > 1.00)) -----------------------------------------------------PhysicalOlapScan[catalog_sales] -------------------------------------------------PhysicalDistribute ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -------------------PhysicalWindow ---------------------PhysicalQuickSort -----------------------PhysicalDistribute -------------------------PhysicalQuickSort ---------------------------PhysicalWindow -----------------------------PhysicalQuickSort -------------------------------PhysicalDistribute ---------------------------------PhysicalQuickSort -----------------------------------PhysicalProject -------------------------------------hashAgg[GLOBAL] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------PhysicalOlapScan[date_dim] +--------PhysicalProject +----------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalWindow +------------------PhysicalQuickSort +--------------------PhysicalDistribute +----------------------PhysicalQuickSort +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](sts.ss_ticket_number = sr.sr_ticket_number)(item = sr.sr_item_sk) +------------------------------------PhysicalProject +--------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------PhysicalOlapScan[store_returns] +------------------------------------hashJoin[INNER_JOIN](sts.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------filter((sts.ss_quantity > 0)(sts.ss_net_profit > 1.00)(sts.ss_net_paid > 0.00)) +------------------------------------------PhysicalOlapScan[store_sales] --------------------------------------PhysicalDistribute -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------hashJoin[INNER_JOIN](sts.ss_ticket_number = sr.sr_ticket_number)(item = sr.sr_item_sk) -----------------------------------------------PhysicalProject -------------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------------hashJoin[INNER_JOIN](sts.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------------------PhysicalProject ---------------------------------------------------filter((sts.ss_quantity > 0)(sts.ss_net_profit > 1.00)(sts.ss_net_paid > 0.00)) -----------------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------------PhysicalDistribute ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out index a3324d16ac2cca..eb45d441220307 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out @@ -17,10 +17,12 @@ PhysicalTopN ----------------------------hashJoin[INNER_JOIN](salesreturns.store_sk = store.s_store_sk) ------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) --------------------------------PhysicalUnion -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_returns] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_returns] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((date_dim.d_date <= 2000-09-02)(date_dim.d_date >= 2000-08-19)) @@ -36,10 +38,12 @@ PhysicalTopN ----------------------------hashJoin[INNER_JOIN](salesreturns.page_sk = catalog_page.cp_catalog_page_sk) ------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) --------------------------------PhysicalUnion -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_returns] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[catalog_returns] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) @@ -55,14 +59,16 @@ PhysicalTopN ----------------------------hashJoin[INNER_JOIN](salesreturns.wsr_web_site_sk = web_site.web_site_sk) ------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) --------------------------------PhysicalUnion -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = web_sales.ws_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_returns] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = web_sales.ws_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[web_sales] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[web_returns] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out index 50af4041a882df..66c68bed08495c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out @@ -9,36 +9,34 @@ PhysicalTopN ------------PhysicalDistribute --------------PhysicalProject ----------------hashJoin[FULL_OUTER_JOIN](web.item_sk = store.item_sk)(web.d_date = store.d_date) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalWindow -------------------------PhysicalQuickSort ---------------------------PhysicalDistribute -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------PhysicalProject +--------------------PhysicalWindow +----------------------PhysicalQuickSort +------------------------PhysicalDistribute +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) ---------------------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalWindow -------------------------PhysicalQuickSort ---------------------------PhysicalDistribute -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) +------------------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------PhysicalWindow +----------------------PhysicalQuickSort +------------------------PhysicalDistribute +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out index fb5bae3e9a7720..a8b0c780ebac46 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 @@ -41,10 +41,12 @@ PhysicalTopN ----------------------------------------------------------------PhysicalProject ------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.item_sk = item.i_item_sk) --------------------------------------------------------------------PhysicalUnion -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------PhysicalOlapScan[web_sales] +----------------------------------------------------------------------PhysicalDistribute +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------------------------------------------PhysicalDistribute +------------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------------PhysicalOlapScan[web_sales] --------------------------------------------------------------------PhysicalDistribute ----------------------------------------------------------------------PhysicalProject ------------------------------------------------------------------------filter((cast(i_class as VARCHAR(*)) = 'maternity')(cast(i_category as VARCHAR(*)) = 'Women')) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out index 410d72ab3a73fd..0e6f011f607f99 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out @@ -1,91 +1,93 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_56 -- PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] +--PhysicalTopN +----hashAgg[LOCAL] +------PhysicalUnion --------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +----------------------PhysicalDistribute +------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject +--------------------------------PhysicalProject +----------------------------------filter(i_color IN ('powder', 'green', 'cyan')) ------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((customer_address.ca_gmt_offset = -6.00)) +----------------------------PhysicalOlapScan[customer_address] +--------PhysicalDistribute +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +----------------------PhysicalDistribute +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) --------------------------PhysicalProject -----------------------------filter((customer_address.ca_gmt_offset = -6.00)) -------------------------------PhysicalOlapScan[customer_address] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject +--------------------------------PhysicalProject +----------------------------------filter(i_color IN ('powder', 'green', 'cyan')) ------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalProject -----------------------------filter((customer_address.ca_gmt_offset = -6.00)) -------------------------------PhysicalOlapScan[customer_address] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((customer_address.ca_gmt_offset = -6.00)) +----------------------------PhysicalOlapScan[customer_address] +--------PhysicalDistribute +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +------------------------filter((customer_address.ca_gmt_offset = -6.00)) +--------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalDistribute +------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) --------------------------PhysicalProject -----------------------------filter((customer_address.ca_gmt_offset = -6.00)) -------------------------------PhysicalOlapScan[customer_address] +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject +--------------------------------PhysicalProject +----------------------------------filter(i_color IN ('powder', 'green', 'cyan')) ------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out index 983aa5cd7e505a..6138c0d491711e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out @@ -5,31 +5,30 @@ CteAnchor[cteId= ( CTEId#0=] ) ----PhysicalProject ------PhysicalWindow --------PhysicalQuickSort -----------PhysicalDistribute -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1)))) -------------------------------------------PhysicalOlapScan[date_dim] +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](call_center.cc_call_center_sk = catalog_sales.cs_call_center_sk) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[call_center] +--------------------------------------filter((((date_dim.d_year = 1999) OR ((date_dim.d_year = 1998) AND (date_dim.d_moy = 12))) OR ((date_dim.d_year = 2000) AND (date_dim.d_moy = 1)))) +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[call_center] --PhysicalProject ----PhysicalTopN ------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out index 1428af1a8b71bb..7ffaac876a79d8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out @@ -33,63 +33,61 @@ PhysicalTopN ----------------------------------PhysicalProject ------------------------------------filter((date_dim.d_date = 2001-03-24)) --------------------------------------PhysicalOlapScan[date_dim] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +----------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalDistribute -----------------------------------PhysicalAssertNumRows -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date = 2001-03-24)) -------------------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalAssertNumRows ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------filter((date_dim.d_date = 2001-03-24)) +----------------------------------------PhysicalOlapScan[date_dim] +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalAssertNumRows ----------------------------------PhysicalDistribute -------------------------------------PhysicalAssertNumRows ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_date = 2001-03-24)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_date = 2001-03-24)) +----------------------------------------PhysicalOlapScan[date_dim] 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 d5751c034cbb83..29b5a1055b46a1 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 @@ -31,16 +31,18 @@ PhysicalTopN ------------------------------------------------PhysicalProject --------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 3)) ----------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------hashJoin[INNER_JOIN](a.ca_address_sk = c.c_current_addr_sk) ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer_address] ------------------PhysicalDistribute --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out index 57ee76f897a7de..f428a69ea3f2c1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out @@ -1,91 +1,94 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_60 -- PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] +--PhysicalTopN +----hashAgg[LOCAL] +------PhysicalUnion --------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) -----------------------------PhysicalDistribute +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((customer_address.ca_gmt_offset = -7.00)) ---------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((customer_address.ca_gmt_offset = -7.00)) ---------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalDistribute +------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject +----------------------------PhysicalProject +------------------------------filter((item.i_category = 'Children')) --------------------------------PhysicalOlapScan[item] +--------PhysicalDistribute +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalProject -------------------------------filter((customer_address.ca_gmt_offset = -7.00)) ---------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalDistribute +------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject +----------------------------PhysicalProject +------------------------------filter((item.i_category = 'Children')) --------------------------------PhysicalOlapScan[item] +--------PhysicalDistribute +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] +----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalDistribute +------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((item.i_category = 'Children')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out index 64a95334d592a9..0cf4b87d09e1f1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out @@ -39,6 +39,7 @@ PhysicalTopN ----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalOlapScan[store] -----------PhysicalProject -------------PhysicalOlapScan[item] +----------PhysicalDistribute +------------PhysicalProject +--------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out index e80fc9bdd9b46d..96c46db256bd3a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out @@ -1,64 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] +--PhysicalTopN +----hashAgg[LOCAL] +------PhysicalUnion --------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) +----------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) -----------------------------------PhysicalOlapScan[time_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[warehouse] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_time_sk = time_dim.t_time_sk) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------PhysicalOlapScan[ship_mode] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) +------------------------------PhysicalOlapScan[time_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[warehouse] +--------PhysicalDistribute +----------PhysicalProject +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_time_sk = time_dim.t_time_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------PhysicalOlapScan[ship_mode] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) -----------------------------------PhysicalOlapScan[time_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[warehouse] +--------------------------------filter((date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) +------------------------------PhysicalOlapScan[time_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out index 1d979be9e1194d..69cc851c1d3eb9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out @@ -27,6 +27,7 @@ PhysicalTopN ----------------------PhysicalProject ------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) --------------------------PhysicalOlapScan[promotion] -----------------PhysicalProject -------------------PhysicalOlapScan[item] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out index e5ec4312174f04..650c95b66811dd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out @@ -11,30 +11,33 @@ PhysicalQuickSort ----------------hashJoin[INNER_JOIN](tmp.time_sk = time_dim.t_time_sk) ------------------hashJoin[INNER_JOIN](tmp.sold_item_sk = item.i_item_sk) --------------------PhysicalUnion -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] ---------------------------PhysicalDistribute +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) ----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalDistribute +------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) ----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) ---------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] ---------------------------PhysicalDistribute +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) ----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +----------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((item.i_manager_id = 1)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out index 0a86b61e335905..8d41268532fd8e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out @@ -3,61 +3,62 @@ CteAnchor[cteId= ( CTEId#3=] ) --CteProducer[cteId= ( CTEId#3=] ) ----hashAgg[LOCAL] -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) -------------------PhysicalProject ---------------------PhysicalOlapScan[catalog_returns] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) -----------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((cast(i_category as VARCHAR(*)) = 'Home')) -------------------------------PhysicalOlapScan[item] +------hashAgg[LOCAL] +--------PhysicalUnion +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +----------------PhysicalProject +------------------PhysicalOlapScan[catalog_returns] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) +--------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999))) -----------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) -------------------PhysicalProject ---------------------PhysicalOlapScan[store_returns] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((cast(i_category as VARCHAR(*)) = 'Home')) -------------------------------PhysicalOlapScan[item] +--------------------------filter((cast(i_category as VARCHAR(*)) = 'Home')) +----------------------------PhysicalOlapScan[item] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999))) +--------------------------PhysicalOlapScan[date_dim] +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) +----------------PhysicalProject +------------------PhysicalOlapScan[store_returns] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +--------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999))) -----------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) -------------------PhysicalProject ---------------------PhysicalOlapScan[web_returns] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) -----------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((cast(i_category as VARCHAR(*)) = 'Home')) -------------------------------PhysicalOlapScan[item] +--------------------------filter((cast(i_category as VARCHAR(*)) = 'Home')) +----------------------------PhysicalOlapScan[item] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999))) +--------------------------PhysicalOlapScan[date_dim] +----------PhysicalDistribute +------------PhysicalProject +--------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) +----------------PhysicalProject +------------------PhysicalOlapScan[web_returns] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) +--------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[web_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999))) -----------------------------PhysicalOlapScan[date_dim] +--------------------------filter((cast(i_category as VARCHAR(*)) = 'Home')) +----------------------------PhysicalOlapScan[item] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(((date_dim.d_year = 1998) OR (date_dim.d_year = 1999))) +--------------------------PhysicalOlapScan[date_dim] --PhysicalTopN ----PhysicalDistribute ------PhysicalTopN 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 4be69f2a531bca..b52788c8f2fb72 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,46 +1,46 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] +--PhysicalTopN +----hashAgg[LOCAL] +------PhysicalUnion --------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------filter(ss_hdemo_sk IS NULL) -------------------------PhysicalOlapScan[store_sales] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------filter(ws_bill_addr_sk IS NULL) -------------------------PhysicalOlapScan[web_sales] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------filter(cs_warehouse_sk IS NULL) -------------------------PhysicalOlapScan[catalog_sales] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[date_dim] +----------PhysicalProject +------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------PhysicalProject +------------------filter(ss_hdemo_sk IS NULL) +--------------------PhysicalOlapScan[store_sales] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +--------------PhysicalDistribute +----------------PhysicalProject +------------------PhysicalOlapScan[date_dim] +--------PhysicalDistribute +----------PhysicalProject +------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------PhysicalProject +------------------filter(ws_bill_addr_sk IS NULL) +--------------------PhysicalOlapScan[web_sales] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +--------------PhysicalDistribute +----------------PhysicalProject +------------------PhysicalOlapScan[date_dim] +--------PhysicalDistribute +----------PhysicalProject +------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------PhysicalProject +------------------filter(cs_warehouse_sk IS NULL) +--------------------PhysicalOlapScan[catalog_sales] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +--------------PhysicalDistribute +----------------PhysicalProject +------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out index f0bb45e55c2451..649b78703e9e23 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out @@ -27,23 +27,22 @@ PhysicalTopN ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[store] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_returns.sr_store_sk = store.s_store_sk) -------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_returns] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_returns.sr_store_sk = store.s_store_sk) +----------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_returns] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store] +----------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store] ------------------PhysicalProject --------------------NestedLoopJoin[CROSS_JOIN] ----------------------PhysicalProject @@ -89,21 +88,20 @@ PhysicalTopN ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[web_page] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) -------------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_returns] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) ---------------------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalProject +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) +----------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_returns] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[web_page] +----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_page] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out index a77d62b9519ae7..855043acbed68b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out @@ -25,40 +25,38 @@ PhysicalTopN ------------------------------PhysicalProject --------------------------------filter((date_dim.d_year = 2000)) ----------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------filter(wr_order_number IS NULL) +--------------------------------hashJoin[LEFT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_returns] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_year = 2000)) +----------------------------------PhysicalOlapScan[date_dim] +------------PhysicalProject +--------------hashAgg[GLOBAL] ----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------PhysicalProject +--------------------------filter(cr_order_number IS NULL) +----------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) ------------------------------PhysicalProject ---------------------------------filter(wr_order_number IS NULL) -----------------------------------hashJoin[LEFT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_returns] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_year = 2000)) -------------------------------------PhysicalOlapScan[date_dim] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_returns] +------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter(cr_order_number IS NULL) -------------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_returns] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((date_dim.d_year = 2000)) ---------------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_year = 2000)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out index e3d8df5df12320..b00cd15207f6fc 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out @@ -23,20 +23,22 @@ PhysicalTopN ----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalIntersect -----------------------PhysicalProject -------------------------filter(substring(ca_zip, 1, 5) IN ('47602', '16704', '35863', '28577', '83910', '36201', '58412', '48162', '28055', '41419', '80332', '38607', '77817', '24891', '16226', '18410', '21231', '59345', '13918', '51089', '20317', '17167', '54585', '67881', '78366', '47770', '18360', '51717', '73108', '14440', '21800', '89338', '45859', '65501', '34948', '25973', '73219', '25333', '17291', '10374', '18829', '60736', '82620', '41351', '52094', '19326', '25214', '54207', '40936', '21814', '79077', '25178', '75742', '77454', '30621', '89193', '27369', '41232', '48567', '83041', '71948', '37119', '68341', '14073', '16891', '62878', '49130', '19833', '24286', '27700', '40979', '50412', '81504', '94835', '84844', '71954', '39503', '57649', '18434', '24987', '12350', '86379', '27413', '44529', '98569', '16515', '27287', '24255', '21094', '16005', '56436', '91110', '68293', '56455', '54558', '10298', '83647', '32754', '27052', '51766', '19444', '13869', '45645', '94791', '57631', '20712', '37788', '41807', '46507', '21727', '71836', '81070', '50632', '88086', '63991', '20244', '31655', '51782', '29818', '63792', '68605', '94898', '36430', '57025', '20601', '82080', '33869', '22728', '35834', '29086', '92645', '98584', '98072', '11652', '78093', '57553', '43830', '71144', '53565', '18700', '90209', '71256', '38353', '54364', '28571', '96560', '57839', '56355', '50679', '45266', '84680', '34306', '34972', '48530', '30106', '15371', '92380', '84247', '92292', '68852', '13338', '34594', '82602', '70073', '98069', '85066', '47289', '11686', '98862', '26217', '47529', '63294', '51793', '35926', '24227', '14196', '24594', '32489', '99060', '49472', '43432', '49211', '14312', '88137', '47369', '56877', '20534', '81755', '15794', '12318', '21060', '73134', '41255', '63073', '81003', '73873', '66057', '51184', '51195', '45676', '92696', '70450', '90669', '98338', '25264', '38919', '59226', '58581', '60298', '17895', '19489', '52301', '80846', '95464', '68770', '51634', '19988', '18367', '18421', '11618', '67975', '25494', '41352', '95430', '15734', '62585', '97173', '33773', '10425', '75675', '53535', '17879', '41967', '12197', '67998', '79658', '59130', '72592', '14851', '43933', '68101', '50636', '25717', '71286', '24660', '58058', '72991', '95042', '15543', '33122', '69280', '11912', '59386', '27642', '65177', '17672', '33467', '64592', '36335', '54010', '18767', '63193', '42361', '49254', '33113', '33159', '36479', '59080', '11855', '81963', '31016', '49140', '29392', '41836', '32958', '53163', '13844', '73146', '23952', '65148', '93498', '14530', '46131', '58454', '13376', '13378', '83986', '12320', '17193', '59852', '46081', '98533', '52389', '13086', '68843', '31013', '13261', '60560', '13443', '45533', '83583', '11489', '58218', '19753', '22911', '25115', '86709', '27156', '32669', '13123', '51933', '39214', '41331', '66943', '14155', '69998', '49101', '70070', '35076', '14242', '73021', '59494', '15782', '29752', '37914', '74686', '83086', '34473', '15751', '81084', '49230', '91894', '60624', '17819', '28810', '63180', '56224', '39459', '55233', '75752', '43639', '55349', '86057', '62361', '50788', '31830', '58062', '18218', '85761', '60083', '45484', '21204', '90229', '70041', '41162', '35390', '16364', '39500', '68908', '26689', '52868', '81335', '40146', '11340', '61527', '61794', '71997', '30415', '59004', '29450', '58117', '69952', '33562', '83833', '27385', '61860', '96435', '48333', '23065', '32961', '84919', '61997', '99132', '22815', '56600', '68730', '48017', '95694', '32919', '88217', '27116', '28239', '58032', '18884', '16791', '21343', '97462', '18569', '75660', '15475')) ---------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalProject -------------------------filter((cnt > 10)) ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer_address] -------------------------------------PhysicalDistribute +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(substring(ca_zip, 1, 5) IN ('47602', '16704', '35863', '28577', '83910', '36201', '58412', '48162', '28055', '41419', '80332', '38607', '77817', '24891', '16226', '18410', '21231', '59345', '13918', '51089', '20317', '17167', '54585', '67881', '78366', '47770', '18360', '51717', '73108', '14440', '21800', '89338', '45859', '65501', '34948', '25973', '73219', '25333', '17291', '10374', '18829', '60736', '82620', '41351', '52094', '19326', '25214', '54207', '40936', '21814', '79077', '25178', '75742', '77454', '30621', '89193', '27369', '41232', '48567', '83041', '71948', '37119', '68341', '14073', '16891', '62878', '49130', '19833', '24286', '27700', '40979', '50412', '81504', '94835', '84844', '71954', '39503', '57649', '18434', '24987', '12350', '86379', '27413', '44529', '98569', '16515', '27287', '24255', '21094', '16005', '56436', '91110', '68293', '56455', '54558', '10298', '83647', '32754', '27052', '51766', '19444', '13869', '45645', '94791', '57631', '20712', '37788', '41807', '46507', '21727', '71836', '81070', '50632', '88086', '63991', '20244', '31655', '51782', '29818', '63792', '68605', '94898', '36430', '57025', '20601', '82080', '33869', '22728', '35834', '29086', '92645', '98584', '98072', '11652', '78093', '57553', '43830', '71144', '53565', '18700', '90209', '71256', '38353', '54364', '28571', '96560', '57839', '56355', '50679', '45266', '84680', '34306', '34972', '48530', '30106', '15371', '92380', '84247', '92292', '68852', '13338', '34594', '82602', '70073', '98069', '85066', '47289', '11686', '98862', '26217', '47529', '63294', '51793', '35926', '24227', '14196', '24594', '32489', '99060', '49472', '43432', '49211', '14312', '88137', '47369', '56877', '20534', '81755', '15794', '12318', '21060', '73134', '41255', '63073', '81003', '73873', '66057', '51184', '51195', '45676', '92696', '70450', '90669', '98338', '25264', '38919', '59226', '58581', '60298', '17895', '19489', '52301', '80846', '95464', '68770', '51634', '19988', '18367', '18421', '11618', '67975', '25494', '41352', '95430', '15734', '62585', '97173', '33773', '10425', '75675', '53535', '17879', '41967', '12197', '67998', '79658', '59130', '72592', '14851', '43933', '68101', '50636', '25717', '71286', '24660', '58058', '72991', '95042', '15543', '33122', '69280', '11912', '59386', '27642', '65177', '17672', '33467', '64592', '36335', '54010', '18767', '63193', '42361', '49254', '33113', '33159', '36479', '59080', '11855', '81963', '31016', '49140', '29392', '41836', '32958', '53163', '13844', '73146', '23952', '65148', '93498', '14530', '46131', '58454', '13376', '13378', '83986', '12320', '17193', '59852', '46081', '98533', '52389', '13086', '68843', '31013', '13261', '60560', '13443', '45533', '83583', '11489', '58218', '19753', '22911', '25115', '86709', '27156', '32669', '13123', '51933', '39214', '41331', '66943', '14155', '69998', '49101', '70070', '35076', '14242', '73021', '59494', '15782', '29752', '37914', '74686', '83086', '34473', '15751', '81084', '49230', '91894', '60624', '17819', '28810', '63180', '56224', '39459', '55233', '75752', '43639', '55349', '86057', '62361', '50788', '31830', '58062', '18218', '85761', '60083', '45484', '21204', '90229', '70041', '41162', '35390', '16364', '39500', '68908', '26689', '52868', '81335', '40146', '11340', '61527', '61794', '71997', '30415', '59004', '29450', '58117', '69952', '33562', '83833', '27385', '61860', '96435', '48333', '23065', '32961', '84919', '61997', '99132', '22815', '56600', '68730', '48017', '95694', '32919', '88217', '27116', '28239', '58032', '18884', '16791', '21343', '97462', '18569', '75660', '15475')) +----------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((cnt > 10)) +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) --------------------------------------PhysicalProject -----------------------------------------filter((cast(c_preferred_cust_flag as VARCHAR(*)) = 'Y')) -------------------------------------------PhysicalOlapScan[customer] +----------------------------------------PhysicalOlapScan[customer_address] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((cast(c_preferred_cust_flag as VARCHAR(*)) = 'Y')) +--------------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out index 02276e4934ad46..44ad4b9321b4e9 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 @@ -16,8 +16,9 @@ CteAnchor[cteId= ( CTEId#2=] ) ----------------------PhysicalProject ------------------------filter((date_dim.d_year = 2002)) --------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------PhysicalOlapScan[customer_address] +--------------PhysicalDistribute +----------------PhysicalProject +------------------PhysicalOlapScan[customer_address] --PhysicalTopN ----PhysicalDistribute ------PhysicalTopN diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out index e1b0093e97affc..67d4702363e19b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out @@ -31,59 +31,57 @@ PhysicalTopN ------------------------------PhysicalProject --------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) ----------------------------------PhysicalOlapScan[date_dim] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) -------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_returns] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] +----------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_returns] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +----------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[date_dim] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_returns] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject +----------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) ------------------------------------PhysicalOlapScan[date_dim] +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_returns] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -----------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) ------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) -----------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out index 5b80e6754b0e6f..fee94a11417d07 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out @@ -19,17 +19,16 @@ PhysicalLimit ----------------------------PhysicalProject ------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) --------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] +--------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) -----------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out index d196ba817e76c7..ae50570ecf54f2 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out @@ -14,6 +14,7 @@ PhysicalQuickSort ----------------------PhysicalProject ------------------------filter(( not (o_comment like '%special%requests%'))) --------------------------PhysicalOlapScan[orders] ---------------------PhysicalProject -----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out index bb74c9cb376ea9..ec4c6370b694a4 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out @@ -10,6 +10,17 @@ PhysicalTopN --------------PhysicalDistribute ----------------PhysicalProject ------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +------------------------PhysicalOlapScan[supplier] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[nation] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((region.r_name = 'EUROPE')) +----------------------------------PhysicalOlapScan[region] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) @@ -18,14 +29,4 @@ PhysicalTopN --------------------------PhysicalProject ----------------------------filter((part.p_size = 15)(p_type like '%BRASS')) ------------------------------PhysicalOlapScan[part] ---------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------------PhysicalOlapScan[supplier] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[nation] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((region.r_name = 'EUROPE')) ---------------------------------PhysicalOlapScan[region] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out index 92f1c8717baec6..2e6728910560ae 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out @@ -22,11 +22,12 @@ PhysicalQuickSort --------------------PhysicalProject ----------------------filter((p_name like 'forest%')) ------------------------PhysicalOlapScan[part] -----------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------PhysicalProject ---------------PhysicalOlapScan[supplier] -------------PhysicalDistribute +----------PhysicalDistribute +------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------PhysicalProject -----------------filter((nation.n_name = 'CANADA')) -------------------PhysicalOlapScan[nation] +----------------PhysicalOlapScan[supplier] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((nation.n_name = 'CANADA')) +--------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out index 4142233ae31535..af742120ca219e 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out @@ -22,11 +22,12 @@ PhysicalQuickSort --------------------PhysicalProject ----------------------filter((p_name like 'forest%')) ------------------------PhysicalOlapScan[part] -----------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------PhysicalProject ---------------PhysicalOlapScan[supplier] -------------PhysicalDistribute +----------PhysicalDistribute +------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------PhysicalProject -----------------filter((nation.n_name = 'CANADA')) -------------------PhysicalOlapScan[nation] +----------------PhysicalOlapScan[supplier] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((nation.n_name = 'CANADA')) +--------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out index c41229b490f474..b2399ebd132d2b 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 @@ -7,20 +7,22 @@ PhysicalQuickSort --------PhysicalDistribute ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) +--------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(c_acctbal)) +----------------PhysicalProject +------------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +--------------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[orders] -----------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(c_acctbal)) -------------------PhysicalProject ---------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) -----------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute ---------------------PhysicalAssertNumRows -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) ---------------------------------PhysicalOlapScan[customer] +------------------PhysicalAssertNumRows +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out index 643f478927d9fd..23fc521663607a 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out @@ -17,7 +17,8 @@ PhysicalTopN ----------------------PhysicalProject ------------------------filter((orders.o_orderdate < 1995-03-15)) --------------------------PhysicalOlapScan[orders] ---------------------PhysicalProject -----------------------filter((customer.c_mktsegment = 'BUILDING')) -------------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((customer.c_mktsegment = 'BUILDING')) +--------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out index 9e50fcef747d39..e20942535d3417 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out @@ -21,9 +21,10 @@ PhysicalQuickSort --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[lineitem] ---------------------------------PhysicalProject -----------------------------------filter((p_name like '%green%')) -------------------------------------PhysicalOlapScan[part] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((p_name like '%green%')) +--------------------------------------PhysicalOlapScan[part] ----------------------------PhysicalDistribute ------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------------------------PhysicalProject @@ -31,6 +32,7 @@ PhysicalQuickSort --------------------------------PhysicalDistribute ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[nation] -----------------PhysicalProject -------------------PhysicalOlapScan[partsupp] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[partsupp] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out index d196ba817e76c7..ae50570ecf54f2 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out @@ -14,6 +14,7 @@ PhysicalQuickSort ----------------------PhysicalProject ------------------------filter(( not (o_comment like '%special%requests%'))) --------------------------PhysicalOlapScan[orders] ---------------------PhysicalProject -----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out index bb74c9cb376ea9..ec4c6370b694a4 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out @@ -10,6 +10,17 @@ PhysicalTopN --------------PhysicalDistribute ----------------PhysicalProject ------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +------------------------PhysicalOlapScan[supplier] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[nation] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((region.r_name = 'EUROPE')) +----------------------------------PhysicalOlapScan[region] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) @@ -18,14 +29,4 @@ PhysicalTopN --------------------------PhysicalProject ----------------------------filter((part.p_size = 15)(p_type like '%BRASS')) ------------------------------PhysicalOlapScan[part] ---------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -----------------------PhysicalOlapScan[supplier] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[nation] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((region.r_name = 'EUROPE')) ---------------------------------PhysicalOlapScan[region] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out index 4142233ae31535..af742120ca219e 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out @@ -22,11 +22,12 @@ PhysicalQuickSort --------------------PhysicalProject ----------------------filter((p_name like 'forest%')) ------------------------PhysicalOlapScan[part] -----------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------PhysicalProject ---------------PhysicalOlapScan[supplier] -------------PhysicalDistribute +----------PhysicalDistribute +------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------PhysicalProject -----------------filter((nation.n_name = 'CANADA')) -------------------PhysicalOlapScan[nation] +----------------PhysicalOlapScan[supplier] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((nation.n_name = 'CANADA')) +--------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out index c41229b490f474..b2399ebd132d2b 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out @@ -7,20 +7,22 @@ PhysicalQuickSort --------PhysicalDistribute ----------hashAgg[LOCAL] ------------PhysicalProject ---------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) +--------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(c_acctbal)) +----------------PhysicalProject +------------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[orders] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +--------------------------PhysicalOlapScan[customer] ----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[orders] -----------------NestedLoopJoin[INNER_JOIN](cast(c_acctbal as DECIMALV3(38, 4)) > avg(c_acctbal)) -------------------PhysicalProject ---------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) -----------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute ---------------------PhysicalAssertNumRows -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) ---------------------------------PhysicalOlapScan[customer] +------------------PhysicalAssertNumRows +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out index 643f478927d9fd..23fc521663607a 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out @@ -17,7 +17,8 @@ PhysicalTopN ----------------------PhysicalProject ------------------------filter((orders.o_orderdate < 1995-03-15)) --------------------------PhysicalOlapScan[orders] ---------------------PhysicalProject -----------------------filter((customer.c_mktsegment = 'BUILDING')) -------------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((customer.c_mktsegment = 'BUILDING')) +--------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out index 05904f5c0626f9..6a73f8f1f00996 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out @@ -30,6 +30,7 @@ PhysicalQuickSort ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[nation] -----------------PhysicalProject -------------------PhysicalOlapScan[partsupp] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[partsupp] diff --git a/regression-test/suites/nereids_syntax_p0/sub_query_correlated.groovy b/regression-test/suites/nereids_syntax_p0/sub_query_correlated.groovy index 98ff46cec7d79e..665ad67e4806ba 100644 --- a/regression-test/suites/nereids_syntax_p0/sub_query_correlated.groovy +++ b/regression-test/suites/nereids_syntax_p0/sub_query_correlated.groovy @@ -22,14 +22,6 @@ suite ("sub_query_correlated") { SET enable_nereids_planner=true """ - sql """ - SET enable_bucket_shuffle_join=false - """ - - sql """ - SET disable_colocate_plan=true - """ - sql """ DROP TABLE IF EXISTS `sub_query_correlated_subquery1` """ diff --git a/regression-test/suites/nereids_syntax_p0/sub_query_diff_old_optimize.groovy b/regression-test/suites/nereids_syntax_p0/sub_query_diff_old_optimize.groovy index 6d5bc11a5a8ef4..4b07a3d5391370 100644 --- a/regression-test/suites/nereids_syntax_p0/sub_query_diff_old_optimize.groovy +++ b/regression-test/suites/nereids_syntax_p0/sub_query_diff_old_optimize.groovy @@ -21,14 +21,6 @@ suite ("sub_query_diff_old_optimize") { SET enable_nereids_planner=true """ - sql """ - SET enable_bucket_shuffle_join=false - """ - - sql """ - SET disable_colocate_plan=true - """ - sql """ DROP TABLE IF EXISTS `sub_query_diff_old_optimize_subquery1` """