From e11e6fe135df3849e77dbf324eaede0eb2dbcea5 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Mon, 24 Jul 2023 15:36:33 +0800 Subject: [PATCH 1/8] [refactor](Nereids) push down all non-slot order key in sort and prune them upper sort (#22034) According the implementation in execution engine, all order keys in SortNode will be output. We must normalize LogicalSort follow by it. We push down all non-slot order key in sort to materialize them behind sort. So, all order key will be slot and do not need do projection by SortNode itself. This will simplify translation of SortNode by avoid to generate resolvedTupleExprs and sortTupleDesc. --- .../java/org/apache/doris/analysis/Expr.java | 10 + .../translator/PhysicalPlanTranslator.java | 245 +++++++----------- .../nereids/rules/rewrite/NormalizeSort.java | 47 +++- .../trees/plans/logical/LogicalSort.java | 35 +-- .../doris/planner/PartitionSortNode.java | 76 +----- .../org/apache/doris/planner/SortNode.java | 28 -- .../shape/query47.out | 4 +- .../shape/query57.out | 4 +- 8 files changed, 150 insertions(+), 299 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java index c198375b0ef05f..5eb4af9dfc2ab2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/Expr.java @@ -722,6 +722,16 @@ public static boolean containsAggregate(List in return false; } + public static void extractSlots(Expr root, Set slotIdSet) { + if (root instanceof SlotRef) { + slotIdSet.add(((SlotRef) root).getDesc().getId()); + return; + } + for (Expr child : root.getChildren()) { + extractSlots(child, slotIdSet); + } + } + /** * Returns an analyzed clone of 'this' with exprs substituted according to smap. * Removes implicit casts and analysis state while cloning/substituting exprs within 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 4b460e944d3ffe..a517b6e5980ce8 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 @@ -66,7 +66,6 @@ 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.EqualTo; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -177,6 +176,7 @@ import com.google.common.collect.Lists; import com.google.common.collect.Maps; import com.google.common.collect.Sets; +import org.apache.commons.collections.CollectionUtils; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -1422,12 +1422,11 @@ public PlanFragment visitPhysicalProject(PhysicalProject project PlanFragment inputFragment = project.child(0).accept(this, context); - List execExprList = project.getProjects() + List projectionExprs = project.getProjects() .stream() .map(e -> ExpressionTranslator.translate(e, context)) .collect(Collectors.toList()); - // TODO: fix the project alias of an aliased relation. - List slotList = project.getProjects() + List slots = project.getProjects() .stream() .map(NamedExpression::toSlot) .collect(Collectors.toList()); @@ -1437,45 +1436,45 @@ public PlanFragment visitPhysicalProject(PhysicalProject project MultiCastDataSink multiCastDataSink = (MultiCastDataSink) inputFragment.getSink(); DataStreamSink dataStreamSink = multiCastDataSink.getDataStreamSinks().get( multiCastDataSink.getDataStreamSinks().size() - 1); - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, null, context); - dataStreamSink.setProjections(execExprList); - dataStreamSink.setOutputTupleDesc(tupleDescriptor); + TupleDescriptor projectionTuple = generateTupleDesc(slots, null, context); + dataStreamSink.setProjections(projectionExprs); + dataStreamSink.setOutputTupleDesc(projectionTuple); return inputFragment; } PlanNode inputPlanNode = inputFragment.getPlanRoot(); - List predicateList = inputPlanNode.getConjuncts(); + List conjuncts = inputPlanNode.getConjuncts(); Set requiredSlotIdSet = Sets.newHashSet(); - for (Expr expr : execExprList) { - extractExecSlot(expr, requiredSlotIdSet); + for (Expr expr : projectionExprs) { + Expr.extractSlots(expr, requiredSlotIdSet); } Set requiredByProjectSlotIdSet = Sets.newHashSet(requiredSlotIdSet); - for (Expr expr : predicateList) { - extractExecSlot(expr, requiredSlotIdSet); + for (Expr expr : conjuncts) { + Expr.extractSlots(expr, requiredSlotIdSet); } // For hash join node, use vSrcToOutputSMap to describe the expression calculation, use // vIntermediateTupleDescList as input, and set vOutputTupleDesc as the final output. // TODO: HashJoinNode's be implementation is not support projection yet, remove this after when supported. if (inputPlanNode instanceof JoinNodeBase) { - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, null, context); - JoinNodeBase hashJoinNode = (JoinNodeBase) inputPlanNode; - hashJoinNode.setvOutputTupleDesc(tupleDescriptor); - hashJoinNode.setvSrcToOutputSMap(execExprList); + TupleDescriptor tupleDescriptor = generateTupleDesc(slots, null, context); + JoinNodeBase joinNode = (JoinNodeBase) inputPlanNode; + joinNode.setvOutputTupleDesc(tupleDescriptor); + joinNode.setvSrcToOutputSMap(projectionExprs); // prune the hashOutputSlotIds - if (hashJoinNode instanceof HashJoinNode) { - ((HashJoinNode) hashJoinNode).getHashOutputSlotIds().clear(); + if (joinNode instanceof HashJoinNode) { + ((HashJoinNode) joinNode).getHashOutputSlotIds().clear(); Set requiredExprIds = Sets.newHashSet(); Set requiredOtherConjunctsSlotIdSet = Sets.newHashSet(); - List otherConjuncts = ((HashJoinNode) hashJoinNode).getOtherJoinConjuncts(); + List otherConjuncts = ((HashJoinNode) joinNode).getOtherJoinConjuncts(); for (Expr expr : otherConjuncts) { - extractExecSlot(expr, requiredOtherConjunctsSlotIdSet); + Expr.extractSlots(expr, requiredOtherConjunctsSlotIdSet); } requiredOtherConjunctsSlotIdSet.forEach(e -> requiredExprIds.add(context.findExprId(e))); requiredSlotIdSet.forEach(e -> requiredExprIds.add(context.findExprId(e))); for (ExprId exprId : requiredExprIds) { - SlotId slotId = ((HashJoinNode) hashJoinNode).getHashOutputExprSlotIdMap().get(exprId); + SlotId slotId = ((HashJoinNode) joinNode).getHashOutputExprSlotIdMap().get(exprId); Preconditions.checkState(slotId != null); - ((HashJoinNode) hashJoinNode).addSlotIdToHashOutputSlotIds(slotId); + ((HashJoinNode) joinNode).addSlotIdToHashOutputSlotIds(slotId); } } return inputFragment; @@ -1487,42 +1486,49 @@ public PlanFragment visitPhysicalProject(PhysicalProject project } if (inputPlanNode instanceof ScanNode) { - TupleDescriptor tupleDescriptor = null; + TupleDescriptor projectionTuple = null; + // slotIdsByOrder is used to ensure the ScanNode's output order is same with current Project + // if we change the output order in translate project, the upper node will receive wrong order + // tuple, since they get the order from project.getOutput() not scan.getOutput()./ + List slotIdsByOrder = Lists.newArrayList(); if (requiredByProjectSlotIdSet.size() != requiredSlotIdSet.size() - || new HashSet<>(execExprList).size() != execExprList.size() - || execExprList.stream().anyMatch(expr -> !(expr instanceof SlotRef))) { - tupleDescriptor = generateTupleDesc(slotList, null, context); - inputPlanNode.setProjectList(execExprList); - inputPlanNode.setOutputTupleDesc(tupleDescriptor); + || new HashSet<>(projectionExprs).size() != projectionExprs.size() + || projectionExprs.stream().anyMatch(expr -> !(expr instanceof SlotRef))) { + projectionTuple = generateTupleDesc(slots, null, context); + inputPlanNode.setProjectList(projectionExprs); + inputPlanNode.setOutputTupleDesc(projectionTuple); } else { - for (int i = 0; i < slotList.size(); ++i) { - context.addExprIdSlotRefPair(slotList.get(i).getExprId(), - (SlotRef) execExprList.get(i)); + for (int i = 0; i < slots.size(); ++i) { + context.addExprIdSlotRefPair(slots.get(i).getExprId(), + (SlotRef) projectionExprs.get(i)); + slotIdsByOrder.add(((SlotRef) projectionExprs.get(i)).getSlotId()); } } // TODO: this is a temporary scheme to support two phase read when has project. // we need to refactor all topn opt into rbo stage. if (inputPlanNode instanceof OlapScanNode) { - ArrayList slots = + ArrayList olapScanSlots = context.getTupleDesc(inputPlanNode.getTupleIds().get(0)).getSlots(); - SlotDescriptor lastSlot = slots.get(slots.size() - 1); + SlotDescriptor lastSlot = olapScanSlots.get(olapScanSlots.size() - 1); if (lastSlot.getColumn() != null && lastSlot.getColumn().getName().equals(Column.ROWID_COL)) { - if (tupleDescriptor != null) { - injectRowIdColumnSlot(tupleDescriptor); + if (projectionTuple != null) { + injectRowIdColumnSlot(projectionTuple); SlotRef slotRef = new SlotRef(lastSlot); inputPlanNode.getProjectList().add(slotRef); requiredByProjectSlotIdSet.add(lastSlot.getId()); + } else { + slotIdsByOrder.add(lastSlot.getId()); } requiredSlotIdSet.add(lastSlot.getId()); } } - updateChildSlotsMaterialization(inputPlanNode, requiredSlotIdSet, - requiredByProjectSlotIdSet, context); + updateScanSlotsMaterialization((ScanNode) inputPlanNode, requiredSlotIdSet, + requiredByProjectSlotIdSet, slotIdsByOrder, context); } else { - TupleDescriptor tupleDescriptor = generateTupleDesc(slotList, null, context); - inputPlanNode.setProjectList(execExprList); + TupleDescriptor tupleDescriptor = generateTupleDesc(slots, null, context); + inputPlanNode.setProjectList(projectionExprs); inputPlanNode.setOutputTupleDesc(tupleDescriptor); } return inputFragment; @@ -1846,34 +1852,24 @@ public PlanFragment visitPhysicalWindow(PhysicalWindow physicalW * ******************************************************************************************** */ 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()); + PlanNode childNode, PlanTranslatorContext context) { + TupleDescriptor sortTuple = generateTupleDesc(partitionTopN.child().getOutput(), null, context); + List orderingExprs = Lists.newArrayList(); + List ascOrders = Lists.newArrayList(); + List nullsFirstParams = Lists.newArrayList(); + List orderKeys = partitionTopN.getOrderKeys(); + orderKeys.forEach(k -> { + orderingExprs.add(ExpressionTranslator.translate(k.getExpr(), context)); + ascOrders.add(k.isAsc()); + nullsFirstParams.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); - + SortInfo sortInfo = new SortInfo(orderingExprs, ascOrders, nullsFirstParams, sortTuple); PartitionSortNode partitionSortNode = new PartitionSortNode(context.nextPlanNodeId(), childNode, partitionTopN.getFunction(), partitionExprs, sortInfo, partitionTopN.hasGlobalLimit(), - partitionTopN.getPartitionLimit(), sortTupleOutputList, oldOrderingExprList); - + partitionTopN.getPartitionLimit()); if (partitionTopN.getStats() != null) { partitionSortNode.setCardinality((long) partitionTopN.getStats().getRowCount()); } @@ -1883,33 +1879,23 @@ private PartitionSortNode translatePartitionSortNode(PhysicalPartitionTopN 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()); + TupleDescriptor sortTuple = generateTupleDesc(sort.child().getOutput(), null, context); + List orderingExprs = Lists.newArrayList(); + List ascOrders = Lists.newArrayList(); + List nullsFirstParams = Lists.newArrayList(); + List orderKeys = sort.getOrderKeys(); + orderKeys.forEach(k -> { + orderingExprs.add(ExpressionTranslator.translate(k.getExpr(), context)); + ascOrders.add(k.isAsc()); + nullsFirstParams.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); + SortInfo sortInfo = new SortInfo(orderingExprs, ascOrders, nullsFirstParams, sortTuple); + SortNode sortNode = new SortNode(context.nextPlanNodeId(), childNode, sortInfo, sort instanceof PhysicalTopN); 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); + SlotDescriptor childRowIdDesc = sortTuple.getSlots().get(sortTuple.getSlots().size() - 1); sortNode.getResolvedTupleExprs().add(new SlotRef(childRowIdDesc)); } if (sort.getStats() != null) { @@ -1919,34 +1905,32 @@ private SortNode translateSortNode(AbstractPhysicalSort sort, Pl return sortNode; } - private void updateChildSlotsMaterialization(PlanNode execPlan, + private void updateScanSlotsMaterialization(ScanNode scanNode, 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); + List slotIdsByOrder, PlanTranslatorContext context) { + // TODO: use smallest slot if do not need any slot in upper node + SlotDescriptor smallest = scanNode.getTupleDesc().getSlots().get(0); + if (CollectionUtils.isNotEmpty(slotIdsByOrder)) { + // if we eliminate project above scan, we should ensure the slot order of scan's output is same with + // the projection's output. So, we need to reorder the output slot in scan's tuple. + Map idToSlotDescMap = scanNode.getTupleDesc().getSlots().stream() + .filter(s -> requiredSlotIdSet.contains(s.getId())) + .collect(Collectors.toMap(SlotDescriptor::getId, s -> s)); + scanNode.getTupleDesc().getSlots().clear(); + for (SlotId slotId : slotIdsByOrder) { + scanNode.getTupleDesc().getSlots().add(idToSlotDescMap.get(slotId)); } + } else { + scanNode.getTupleDesc().getSlots().removeIf(s -> !requiredSlotIdSet.contains(s.getId())); + } + if (scanNode.getTupleDesc().getSlots().isEmpty()) { + scanNode.getTupleDesc().getSlots().add(smallest); + } + try { + scanNode.updateRequiredSlots(context, requiredByProjectSlotIdSet); + } catch (UserException e) { + Util.logAndThrowRuntimeException(LOG, + "User Exception while reset external file scan node contexts.", e); } } @@ -1959,16 +1943,6 @@ private void addConjunctsToPlanNode(PhysicalFilter filter, updateLegacyPlanIdToPhysicalPlan(planNode, filter); } - private void extractExecSlot(Expr root, Set slotIdList) { - if (root instanceof SlotRef) { - slotIdList.add(((SlotRef) root).getDesc().getId()); - return; - } - for (Expr child : root.getChildren()) { - extractExecSlot(child, slotIdList); - } - } - private TupleDescriptor generateTupleDesc(List slotList, TableIf table, Set deferredMaterializedExprIds, PlanTranslatorContext context) { TupleDescriptor tupleDescriptor = context.generateTupleDesc(); @@ -1991,39 +1965,6 @@ private TupleDescriptor generateTupleDesc(List slotList, TableIf table, Pl return tupleDescriptor; } - private TupleDescriptor generateTupleDesc(List slotList, List orderKeyList, - List newOrderingExprList, - PlanTranslatorContext context, Table table) { - TupleDescriptor tupleDescriptor = context.generateTupleDesc(); - Set alreadyExists = Sets.newHashSet(); - tupleDescriptor.setTable(table); - for (OrderKey orderKey : orderKeyList) { - SlotReference slotReference; - if (orderKey.getExpr() instanceof SlotReference) { - slotReference = (SlotReference) orderKey.getExpr(); - } else { - slotReference = (SlotReference) new Alias(orderKey.getExpr(), orderKey.getExpr().toString()).toSlot(); - } - // TODO: trick here, we need semanticEquals to remove redundant expression - if (alreadyExists.contains(slotReference.getExprId())) { - newOrderingExprList.add(context.findSlotRef(slotReference.getExprId())); - continue; - } - context.createSlotDesc(tupleDescriptor, slotReference); - newOrderingExprList.add(context.findSlotRef(slotReference.getExprId())); - alreadyExists.add(slotReference.getExprId()); - } - for (Slot slot : slotList) { - if (alreadyExists.contains(slot.getExprId())) { - continue; - } - context.createSlotDesc(tupleDescriptor, (SlotReference) slot); - alreadyExists.add(slot.getExprId()); - } - - return tupleDescriptor; - } - private PlanFragment connectJoinNode(HashJoinNode hashJoinNode, PlanFragment leftFragment, PlanFragment rightFragment, PlanTranslatorContext context, AbstractPlan join) { hashJoinNode.setChild(0, leftFragment.getPlanRoot()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeSort.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeSort.java index 9480a1bb835e20..5426b5af6a7a48 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeSort.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/NormalizeSort.java @@ -17,33 +17,52 @@ package org.apache.doris.nereids.rules.rewrite; +import org.apache.doris.nereids.properties.OrderKey; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; -import java.util.stream.Collectors; +import java.util.List; +import java.util.stream.Stream; /** - * the sort node will create new slots for order by keys if the order by keys is not in the output - * so need create a project above sort node to prune the unnecessary order by keys. This means the - * Tuple slots size is difference to PhysicalSort.output.size. If not prune and hide the order key, - * the upper plan node will see the temporary slots and treat as output, and then translate failed. - * This is trick, we should add sort output tuple to ensure the tuple slot size is equals, but it - * has large workload. I think we should refactor the PhysicalPlanTranslator in the future, and - * process PhysicalProject(output)/PhysicalDistribute more general. + * SortNode on BE always output order keys because BE needs them to do merge sort. So we normalize LogicalSort as BE + * expected to materialize order key before sort by bottom project and then prune the useless column after sort by + * top project. */ public class NormalizeSort extends OneRewriteRuleFactory { @Override public Rule build() { - return logicalSort() - .when(sort -> !sort.isNormalized() && !sort.getOutputSet() - .containsAll(sort.getOrderKeys().stream() - .map(orderKey -> orderKey.getExpr()).collect(Collectors.toSet()))) + return logicalSort().whenNot(sort -> sort.getOrderKeys().stream() + .map(OrderKey::getExpr).allMatch(Slot.class::isInstance)) .then(sort -> { - return new LogicalProject(sort.getOutput(), ImmutableList.of(), false, - sort.withNormalize(true)); + List newProjects = Lists.newArrayList(); + List newOrderKeys = sort.getOrderKeys().stream() + .map(orderKey -> { + Expression expr = orderKey.getExpr(); + if (!(expr instanceof Slot)) { + Alias alias = new Alias(expr, expr.toSql()); + newProjects.add(alias); + expr = alias.toSlot(); + } + return orderKey.withExpression(expr); + }).collect(ImmutableList.toImmutableList()); + List bottomProjections = Stream.concat( + sort.child().getOutput().stream(), + newProjects.stream() + ).collect(ImmutableList.toImmutableList()); + List topProjections = sort.getOutput().stream() + .map(NamedExpression.class::cast) + .collect(ImmutableList.toImmutableList()); + return new LogicalProject<>(topProjections, sort.withOrderKeysAndChild(newOrderKeys, + new LogicalProject<>(bottomProjections, sort.child()))); }).toRule(RuleType.NORMALIZE_SORT); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSort.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSort.java index 5918422966fea2..f139656f01c925 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSort.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSort.java @@ -46,29 +46,17 @@ public class LogicalSort extends LogicalUnary orderKeys; - private final boolean normalized; - public LogicalSort(List orderKeys, CHILD_TYPE child) { this(orderKeys, Optional.empty(), Optional.empty(), child); } - public LogicalSort(List orderKeys, CHILD_TYPE child, boolean normalized) { - this(orderKeys, Optional.empty(), Optional.empty(), child, normalized); - } - /** * Constructor for LogicalSort. */ public LogicalSort(List orderKeys, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child) { - this(orderKeys, groupExpression, logicalProperties, child, false); - } - - public LogicalSort(List orderKeys, Optional groupExpression, - Optional logicalProperties, CHILD_TYPE child, boolean normalized) { super(PlanType.LOGICAL_SORT, groupExpression, logicalProperties, child); this.orderKeys = ImmutableList.copyOf(Objects.requireNonNull(orderKeys, "orderKeys can not be null")); - this.normalized = normalized; } @Override @@ -80,10 +68,6 @@ public List getOrderKeys() { return orderKeys; } - public boolean isNormalized() { - return normalized; - } - @Override public String toString() { return Utils.toSqlString("LogicalSort[" + id.asInt() + "]", @@ -98,7 +82,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - LogicalSort that = (LogicalSort) o; + LogicalSort that = (LogicalSort) o; return Objects.equals(orderKeys, that.orderKeys); } @@ -122,30 +106,27 @@ public List getExpressions() { @Override public LogicalSort withChildren(List children) { Preconditions.checkArgument(children.size() == 1); - return new LogicalSort<>(orderKeys, children.get(0), normalized); + return new LogicalSort<>(orderKeys, children.get(0)); } @Override public LogicalSort withGroupExpression(Optional groupExpression) { - return new LogicalSort<>(orderKeys, groupExpression, Optional.of(getLogicalProperties()), child(), - normalized); + return new LogicalSort<>(orderKeys, groupExpression, Optional.of(getLogicalProperties()), child()); } @Override - public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + public LogicalSort withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { Preconditions.checkArgument(children.size() == 1); - return new LogicalSort<>(orderKeys, groupExpression, logicalProperties, children.get(0), - normalized); + return new LogicalSort<>(orderKeys, groupExpression, logicalProperties, children.get(0)); } public LogicalSort withOrderKeys(List orderKeys) { return new LogicalSort<>(orderKeys, Optional.empty(), - Optional.of(getLogicalProperties()), child(), false); + Optional.of(getLogicalProperties()), child()); } - public LogicalSort withNormalize(boolean orderKeysPruned) { - return new LogicalSort<>(orderKeys, groupExpression, Optional.of(getLogicalProperties()), child(), - orderKeysPruned); + public LogicalSort withOrderKeysAndChild(List orderKeys, Plan child) { + return new LogicalSort<>(orderKeys, child); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/PartitionSortNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/PartitionSortNode.java index ef198891ccd9df..9e14b4f267de99 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/PartitionSortNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/PartitionSortNode.java @@ -17,12 +17,8 @@ package org.apache.doris.planner; -import org.apache.doris.analysis.Analyzer; import org.apache.doris.analysis.Expr; -import org.apache.doris.analysis.SlotDescriptor; -import org.apache.doris.analysis.SlotId; import org.apache.doris.analysis.SortInfo; -import org.apache.doris.common.NotImplementedException; import org.apache.doris.nereids.trees.plans.WindowFuncType; import org.apache.doris.statistics.StatisticalType; import org.apache.doris.thrift.TExplainLevel; @@ -34,40 +30,29 @@ import com.google.common.base.Joiner; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; -import org.apache.logging.log4j.LogManager; -import org.apache.logging.log4j.Logger; -import java.util.ArrayList; -import java.util.HashSet; import java.util.Iterator; import java.util.List; -import java.util.Set; /** * PartitionSortNode. * PartitionSortNode is only used in the Nereids. */ public class PartitionSortNode extends PlanNode { - private static final Logger LOG = LogManager.getLogger(PartitionSortNode.class); - private List resolvedTupleExprs; private final WindowFuncType function; private final List partitionExprs; private final SortInfo info; private final boolean hasGlobalLimit; private final long partitionLimit; - private boolean isUnusedExprRemoved = false; - private ArrayList nullabilityChangedFlags = Lists.newArrayList(); - /** * Constructor. */ public PartitionSortNode(PlanNodeId id, PlanNode input, WindowFuncType function, List partitionExprs, - SortInfo info, boolean hasGlobalLimit, long partitionLimit, - List outputList, List orderingExpr) { + SortInfo info, boolean hasGlobalLimit, long partitionLimit) { super(id, "PartitionTopN", StatisticalType.PARTITION_TOPN_NODE); + Preconditions.checkArgument(info.getOrderingExprs().size() == info.getIsAscOrder().size()); this.function = function; this.partitionExprs = partitionExprs; this.info = info; @@ -77,38 +62,12 @@ public PartitionSortNode(PlanNodeId id, PlanNode input, WindowFuncType function, this.tblRefIds.addAll(Lists.newArrayList(info.getSortTupleDescriptor().getId())); this.nullableTupleIds.addAll(input.getNullableTupleIds()); this.children.add(input); - - List resolvedTupleExprs = new ArrayList<>(); - for (Expr order : orderingExpr) { - if (!resolvedTupleExprs.contains(order)) { - resolvedTupleExprs.add(order); - } - } - for (Expr output : outputList) { - if (!resolvedTupleExprs.contains(output)) { - resolvedTupleExprs.add(output); - } - } - this.resolvedTupleExprs = ImmutableList.copyOf(resolvedTupleExprs); - info.setSortTupleSlotExprs(resolvedTupleExprs); - - nullabilityChangedFlags.clear(); - for (int i = 0; i < resolvedTupleExprs.size(); i++) { - nullabilityChangedFlags.add(false); - } - Preconditions.checkArgument(info.getOrderingExprs().size() == info.getIsAscOrder().size()); } public SortInfo getSortInfo() { return info; } - @Override - public void getMaterializedIds(Analyzer analyzer, List ids) { - super.getMaterializedIds(analyzer, ids); - Expr.getIds(info.getOrderingExprs(), null, ids); - } - @Override public String getNodeExplainString(String prefix, TExplainLevel detailLevel) { if (detailLevel == TExplainLevel.BRIEF) { @@ -164,34 +123,12 @@ public String getNodeExplainString(String prefix, TExplainLevel detailLevel) { return output.toString(); } - private void removeUnusedExprs() { - if (!isUnusedExprRemoved) { - if (resolvedTupleExprs != null) { - List slotDescriptorList = this.info.getSortTupleDescriptor().getSlots(); - for (int i = slotDescriptorList.size() - 1; i >= 0; i--) { - if (!slotDescriptorList.get(i).isMaterialized()) { - resolvedTupleExprs.remove(i); - nullabilityChangedFlags.remove(i); - } - } - } - isUnusedExprRemoved = true; - } - } - @Override protected void toThrift(TPlanNode msg) { msg.node_type = TPlanNodeType.PARTITION_SORT_NODE; TSortInfo sortInfo = info.toThrift(); Preconditions.checkState(tupleIds.size() == 1, "Incorrect size for tupleIds in PartitionSortNode"); - removeUnusedExprs(); - if (resolvedTupleExprs != null) { - sortInfo.setSortTupleSlotExprs(Expr.treesToThrift(resolvedTupleExprs)); - // FIXME this is a bottom line solution for wrong nullability of resolvedTupleExprs - // remove the following line after nereids online - sortInfo.setSlotExprsNullabilityChangedFlags(nullabilityChangedFlags); - } TopNAlgorithm topNAlgorithm; if (function == WindowFuncType.ROW_NUMBER) { @@ -210,13 +147,4 @@ protected void toThrift(TPlanNode msg) { partitionSortNode.setPartitionInnerLimit(partitionLimit); msg.partition_sort_node = partitionSortNode; } - - @Override - public Set computeInputSlotIds(Analyzer analyzer) throws NotImplementedException { - removeUnusedExprs(); - List materializedTupleExprs = new ArrayList<>(resolvedTupleExprs); - List result = Lists.newArrayList(); - Expr.getIds(materializedTupleExprs, null, result); - return new HashSet<>(result); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/SortNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/SortNode.java index aab8f44186d7f1..c4dbe606444af1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/SortNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/SortNode.java @@ -27,7 +27,6 @@ import org.apache.doris.analysis.SlotId; import org.apache.doris.analysis.SlotRef; import org.apache.doris.analysis.SortInfo; -import org.apache.doris.analysis.TupleDescriptor; import org.apache.doris.common.NotImplementedException; import org.apache.doris.common.UserException; import org.apache.doris.statistics.StatisticalType; @@ -329,31 +328,4 @@ public Set computeInputSlotIds(Analyzer analyzer) throws NotImplementedE Expr.getIds(materializedTupleExprs, null, result); return new HashSet<>(result); } - - /** - * Supplement the information needed by be for the sort node. - * TODO: currently we only process slotref, so when order key is a + 1, we will failed. - */ - public void finalizeForNereids(TupleDescriptor tupleDescriptor, - List outputList, List orderingExpr) { - resolvedTupleExprs = Lists.newArrayList(); - // TODO: should fix the duplicate order by exprs in nereids code later - for (Expr order : orderingExpr) { - if (!resolvedTupleExprs.contains(order)) { - resolvedTupleExprs.add(order); - } - } - for (Expr output : outputList) { - if (!resolvedTupleExprs.contains(output)) { - resolvedTupleExprs.add(output); - } - } - info.setSortTupleDesc(tupleDescriptor); - info.setSortTupleSlotExprs(resolvedTupleExprs); - - nullabilityChangedFlags.clear(); - for (int i = 0; i < resolvedTupleExprs.size(); i++) { - nullabilityChangedFlags.add(false); - } - } } 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 49fc532871b99d..c6f3c083e0651f 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 @@ -34,13 +34,13 @@ CteAnchor[cteId= ( CTEId#0=] ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](s_store_name = v1_lead.s_store_name)(v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) +------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) --------------PhysicalDistribute ----------------PhysicalProject ------------------CteConsumer[cteId= ( CTEId#0=] ) --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](s_store_name = v1_lag.s_store_name)(v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) +------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------CteConsumer[cteId= ( CTEId#0=] ) 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 1b3a0610a49686..ed85d04d8c3a0c 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 @@ -34,13 +34,13 @@ CteAnchor[cteId= ( CTEId#0=] ) ------PhysicalDistribute --------PhysicalTopN ----------PhysicalProject -------------hashJoin[INNER_JOIN](i_brand = v1_lead.i_brand)(v1.i_category = v1_lead.i_category)(v1.cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1)) +------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1)) --------------PhysicalDistribute ----------------PhysicalProject ------------------CteConsumer[cteId= ( CTEId#0=] ) --------------PhysicalDistribute ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](i_brand = v1_lag.i_brand)(v1.i_category = v1_lag.i_category)(v1.cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1)) +------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1)) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------CteConsumer[cteId= ( CTEId#0=] ) From 0b411ebede1757bf1d1b9ee4919068673c27b4ef Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Mon, 24 Jul 2023 20:46:27 +0800 Subject: [PATCH 2/8] [fix](Nereids) translate partition topn order key on wrong tuple (#22168) partition key should on child tuple, sort key should on partition top's tuple --- .../nereids/glue/translator/PhysicalPlanTranslator.java | 7 ++++--- 1 file changed, 4 insertions(+), 3 deletions(-) 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 a517b6e5980ce8..d7157118ebcd8a 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 @@ -1853,6 +1853,10 @@ public PlanFragment visitPhysicalWindow(PhysicalWindow physicalW private PartitionSortNode translatePartitionSortNode(PhysicalPartitionTopN partitionTopN, PlanNode childNode, PlanTranslatorContext context) { + List partitionExprs = partitionTopN.getPartitionKeys().stream() + .map(e -> ExpressionTranslator.translate(e, context)) + .collect(Collectors.toList()); + // partition key should on child tuple, sort key should on partition top's tuple TupleDescriptor sortTuple = generateTupleDesc(partitionTopN.child().getOutput(), null, context); List orderingExprs = Lists.newArrayList(); List ascOrders = Lists.newArrayList(); @@ -1863,9 +1867,6 @@ private PartitionSortNode translatePartitionSortNode(PhysicalPartitionTopN partitionExprs = partitionTopN.getPartitionKeys().stream() - .map(e -> ExpressionTranslator.translate(e, context)) - .collect(Collectors.toList()); SortInfo sortInfo = new SortInfo(orderingExprs, ascOrders, nullsFirstParams, sortTuple); PartitionSortNode partitionSortNode = new PartitionSortNode(context.nextPlanNodeId(), childNode, partitionTopN.getFunction(), partitionExprs, sortInfo, partitionTopN.hasGlobalLimit(), From b175da6b6dfaac3888489e4c7a40306a5dd08f10 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Wed, 26 Jul 2023 11:38:50 +0800 Subject: [PATCH 3/8] [fix](Nereids) translate failed when enable topn two phase opt (#22197) 1. should not add rowid slot to reslovedTupleExprs 2. should set notMaterialize to sort's tuple when do two phase opt --- .../translator/PhysicalPlanTranslator.java | 11 +++++++---- .../processor/post/TwoPhaseReadOpt.java | 4 ++-- .../trees/plans/physical/PhysicalTopN.java | 1 - regression-test/data/nereids_p0/sort/sort.out | 6 ++++++ .../suites/nereids_p0/sort/sort.groovy | 18 ++++++++++++++++++ 5 files changed, 33 insertions(+), 7 deletions(-) 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 d7157118ebcd8a..a4f172ea10a35d 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 @@ -1880,7 +1880,12 @@ private PartitionSortNode translatePartitionSortNode(PhysicalPartitionTopN sort, PlanNode childNode, PlanTranslatorContext context) { - TupleDescriptor sortTuple = generateTupleDesc(sort.child().getOutput(), null, context); + Set deferredMaterializedExprIds = sort + .getMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS) + .map(s -> (Set) s) + .orElse(Collections.emptySet()); + TupleDescriptor sortTuple = generateTupleDesc(sort.child().getOutput(), + null, deferredMaterializedExprIds, context); List orderingExprs = Lists.newArrayList(); List ascOrders = Lists.newArrayList(); List nullsFirstParams = Lists.newArrayList(); @@ -1892,12 +1897,10 @@ private SortNode translateSortNode(AbstractPhysicalSort sort, Pl }); SortInfo sortInfo = new SortInfo(orderingExprs, ascOrders, nullsFirstParams, sortTuple); SortNode sortNode = new SortNode(context.nextPlanNodeId(), childNode, sortInfo, sort instanceof PhysicalTopN); - if (sort.getMutableState(PhysicalTopN.TWO_PHASE_READ_OPT).isPresent()) { + if (sort.getMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS).isPresent()) { sortNode.setUseTwoPhaseReadOpt(true); sortNode.getSortInfo().setUseTwoPhaseRead(); injectRowIdColumnSlot(sortNode.getSortInfo().getSortTupleDescriptor()); - SlotDescriptor childRowIdDesc = sortTuple.getSlots().get(sortTuple.getSlots().size() - 1); - sortNode.getResolvedTupleExprs().add(new SlotRef(childRowIdDesc)); } if (sort.getStats() != null) { sortNode.setCardinality((long) sort.getStats().getRowCount()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/TwoPhaseReadOpt.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/TwoPhaseReadOpt.java index 72d1a7d2db77b9..93dd579e7084a7 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/TwoPhaseReadOpt.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/TwoPhaseReadOpt.java @@ -144,9 +144,9 @@ public PhysicalTopN visitPhysicalTopN(PhysicalTopN mergeTopN, Ca .map(Slot.class::cast) .map(NamedExpression::getExprId) .forEach(deferredMaterializedExprIds::remove); - localTopN.setMutableState(PhysicalTopN.TWO_PHASE_READ_OPT, true); - mergeTopN.setMutableState(PhysicalTopN.TWO_PHASE_READ_OPT, true); olapScan.setMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS, deferredMaterializedExprIds); + localTopN.setMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS, deferredMaterializedExprIds); + mergeTopN.setMutableState(PhysicalOlapScan.DEFERRED_MATERIALIZED_SLOTS, deferredMaterializedExprIds); return mergeTopN; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTopN.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTopN.java index af71f50e6b25f8..0f1d0069b34d4d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTopN.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTopN.java @@ -42,7 +42,6 @@ public class PhysicalTopN extends AbstractPhysicalSort implements TopN { public static final String TOPN_RUNTIME_FILTER = "topn_runtime_filter"; - public static final String TWO_PHASE_READ_OPT = "two_phase_read_opt"; private final long limit; private final long offset; diff --git a/regression-test/data/nereids_p0/sort/sort.out b/regression-test/data/nereids_p0/sort/sort.out index ffddb115bce80e..38326d3e23a779 100644 --- a/regression-test/data/nereids_p0/sort/sort.out +++ b/regression-test/data/nereids_p0/sort/sort.out @@ -50,3 +50,9 @@ true 2023-03-21T07:00 area1 p0 aaaaa ddddd2 100.000 100.000 100.000 100.000 2023-03-21T17:00 2023-03-21T06:00 area1 p0 aaaaa ddddd1 100.000 100.000 100.000 100.000 2023-03-21T17:00 +-- !sql -- +1 1024 +2 1024 +3 0 +4 \N + diff --git a/regression-test/suites/nereids_p0/sort/sort.groovy b/regression-test/suites/nereids_p0/sort/sort.groovy index 11f4bc5dfbaa28..0dfe4a68d7cd00 100644 --- a/regression-test/suites/nereids_p0/sort/sort.groovy +++ b/regression-test/suites/nereids_p0/sort/sort.groovy @@ -89,4 +89,22 @@ suite("sort") { """ qt_sql_orderby_non_overlap_desc """ select * from sort_non_overlap order by time_period desc limit 4; """ + + sql """ DROP TABLE if exists `sort_default_value`; """ + sql """ CREATE TABLE `sort_default_value` ( + `k1` int NOT NULL + ) ENGINE=OLAP + DUPLICATE KEY(`k1`) + DISTRIBUTED BY HASH(`k1`) BUCKETS 1 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "disable_auto_compaction" = "true" + ); + """ + sql "insert into sort_default_value values (1)" + sql "insert into sort_default_value values (2)" + sql """ alter table sort_default_value add column k4 INT default "1024" """ + sql "insert into sort_default_value values (3, 0)" + sql "insert into sort_default_value values (4, null)" + qt_sql "select * from sort_default_value order by k1 limit 10" } From 73377270e5e86391ff45708e170aa789b8146022 Mon Sep 17 00:00:00 2001 From: AKIRA <33112463+Kikyou1997@users.noreply.github.com> Date: Wed, 12 Jul 2023 10:55:40 +0800 Subject: [PATCH 4/8] [opt](nereids) update CTEConsumer's stats when CTEProducer's stats updated (#21469) --- .../apache/doris/nereids/CascadesContext.java | 31 +++++++++++++++++++ .../jobs/cascades/CostAndEnforcerJob.java | 3 +- .../nereids/jobs/cascades/DeriveStatsJob.java | 3 +- .../doris/nereids/stats/StatsCalculator.java | 24 +++++++++----- .../nereids/stats/StatsCalculatorTest.java | 14 ++++----- 5 files changed, 59 insertions(+), 16 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index e16f96c93256ad..2900cc0d8898f5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -21,6 +21,8 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; import org.apache.doris.datasource.CatalogIf; +import org.apache.doris.catalog.Table; +import org.apache.doris.common.Pair; import org.apache.doris.nereids.analyzer.Scope; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.jobs.Job; @@ -35,6 +37,7 @@ import org.apache.doris.nereids.jobs.scheduler.JobStack; import org.apache.doris.nereids.jobs.scheduler.ScheduleContext; import org.apache.doris.nereids.jobs.scheduler.SimpleJobScheduler; +import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.Memo; import org.apache.doris.nereids.processor.post.RuntimeFilterContext; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -45,6 +48,7 @@ import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SubqueryExpr; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; @@ -55,6 +59,8 @@ import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; +import org.apache.doris.statistics.ColumnStatistic; +import org.apache.doris.statistics.Statistics; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -64,6 +70,7 @@ import java.util.HashSet; import java.util.List; import java.util.Map; +import java.util.Map.Entry; import java.util.Optional; import java.util.Set; import java.util.Stack; @@ -104,6 +111,9 @@ public class CascadesContext implements ScheduleContext { private Map> consumerIdToFilters = new HashMap<>(); private Map> cteIdToConsumerUnderProjects = new HashMap<>(); + // Used to update consumer's stats + private Map, Group>>> cteIdToConsumerGroup = new HashMap<>(); + public CascadesContext(Plan plan, Memo memo, StatementContext statementContext, PhysicalProperties requestProperties) { this(plan, memo, statementContext, new CTEContext(), requestProperties); @@ -579,4 +589,25 @@ public boolean couldPruneColumnOnProducer(CTEId cteId) { Set consumerIds = this.cteIdToConsumerUnderProjects.get(cteId); return consumerIds.size() == this.cteIdToConsumers.get(cteId).size(); } + + public void addCTEConsumerGroup(CTEId cteId, Group g, Map producerSlotToConsumerSlot) { + List, Group>> consumerGroups = + this.cteIdToConsumerGroup.computeIfAbsent(cteId, k -> new ArrayList<>()); + consumerGroups.add(Pair.of(producerSlotToConsumerSlot, g)); + } + + /** + * Update CTE consumer group as producer's stats update + */ + public void updateConsumerStats(CTEId cteId, Statistics statistics) { + List, Group>> consumerGroups = this.cteIdToConsumerGroup.get(cteId); + for (Pair, Group> p : consumerGroups) { + Map producerSlotToConsumerSlot = p.first; + Statistics updatedConsumerStats = new Statistics(statistics); + for (Entry entry : statistics.columnStatistics().entrySet()) { + updatedConsumerStats.addColumnStats(producerSlotToConsumerSlot.get(entry.getKey()), entry.getValue()); + } + p.value().setStatistics(updatedConsumerStats); + } + } } 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 37f5e5b6818ac4..1c41a48ecd16aa 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 @@ -245,7 +245,8 @@ private boolean calculateEnforce(List requestChildrenPropert StatsCalculator statsCalculator = StatsCalculator.estimate(groupExpression, context.getCascadesContext().getConnectContext().getSessionVariable().getForbidUnknownColStats(), context.getCascadesContext().getConnectContext().getTotalColumnStatisticMap(), - context.getCascadesContext().getConnectContext().getSessionVariable().isPlayNereidsDump()); + context.getCascadesContext().getConnectContext().getSessionVariable().isPlayNereidsDump(), + context.getCascadesContext()); if (!context.getCascadesContext().getConnectContext().getSessionVariable().isPlayNereidsDump() && context.getCascadesContext().getConnectContext().getSessionVariable().isEnableMinidump()) { context.getCascadesContext().getConnectContext().getTotalColumnStatisticMap() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJob.java index 6ff01f278dec29..cfe952c0f26873 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJob.java @@ -105,7 +105,8 @@ public void execute() { context.getCascadesContext().getConnectContext().getSessionVariable().getForbidUnknownColStats(), context.getCascadesContext().getConnectContext().getTotalColumnStatisticMap(), context.getCascadesContext().getConnectContext().getSessionVariable().isPlayNereidsDump(), - cteIdToStats); + cteIdToStats, + context.getCascadesContext()); STATS_STATE_TRACER.log(StatsStateEvent.of(groupExpression, groupExpression.getOwnerGroup().getStatistics())); if (ConnectContext.get().getSessionVariable().isEnableMinidump() 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 81eb2141c7d77c..5fe4b05c4b2462 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 @@ -25,6 +25,7 @@ import org.apache.doris.common.Config; import org.apache.doris.common.FeConstants; import org.apache.doris.common.Pair; +import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; @@ -161,14 +162,17 @@ public class StatsCalculator extends DefaultPlanVisitor { private Map cteIdToStats; + private CascadesContext cascadesContext; + private StatsCalculator(GroupExpression groupExpression, boolean forbidUnknownColStats, Map columnStatisticMap, boolean isPlayNereidsDump, - Map cteIdToStats) { + Map cteIdToStats, CascadesContext context) { this.groupExpression = groupExpression; this.forbidUnknownColStats = forbidUnknownColStats; this.totalColumnStatisticMap = columnStatisticMap; this.isPlayNereidsDump = isPlayNereidsDump; this.cteIdToStats = Objects.requireNonNull(cteIdToStats, "CTEIdToStats can't be null"); + this.cascadesContext = context; } public Map getTotalHistogramMap() { @@ -192,25 +196,26 @@ public void setTotalColumnStatisticMap(Map totalColumnS */ public static StatsCalculator estimate(GroupExpression groupExpression, boolean forbidUnknownColStats, Map columnStatisticMap, boolean isPlayNereidsDump, - Map cteIdToStats) { + Map cteIdToStats, CascadesContext context) { StatsCalculator statsCalculator = new StatsCalculator( - groupExpression, forbidUnknownColStats, columnStatisticMap, isPlayNereidsDump, cteIdToStats); + groupExpression, forbidUnknownColStats, columnStatisticMap, isPlayNereidsDump, cteIdToStats, context); statsCalculator.estimate(); return statsCalculator; } public static StatsCalculator estimate(GroupExpression groupExpression, boolean forbidUnknownColStats, - Map columnStatisticMap, boolean isPlayNereidsDump) { + Map columnStatisticMap, boolean isPlayNereidsDump, CascadesContext context) { return StatsCalculator.estimate(groupExpression, forbidUnknownColStats, columnStatisticMap, isPlayNereidsDump, - new HashMap<>()); + new HashMap<>(), context); } - public static void estimate(GroupExpression groupExpression) { + // For unit test only + public static void estimate(GroupExpression groupExpression, CascadesContext context) { StatsCalculator statsCalculator = new StatsCalculator(groupExpression, false, - new HashMap<>(), false, Collections.EMPTY_MAP); + new HashMap<>(), false, Collections.EMPTY_MAP, context); statsCalculator.estimate(); } @@ -1013,6 +1018,8 @@ public Statistics visitLogicalCTEProducer(LogicalCTEProducer cte @Override public Statistics visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, Void context) { CTEId cteId = cteConsumer.getCteId(); + cascadesContext.addCTEConsumerGroup(cteConsumer.getCteId(), groupExpression.getOwnerGroup(), + cteConsumer.getProducerToConsumerOutputMap()); Statistics prodStats = cteIdToStats.get(cteId); Preconditions.checkArgument(prodStats != null, String.format("Stats for CTE: %s not found", cteId)); Statistics consumerStats = new Statistics(prodStats.getRowCount(), new HashMap<>()); @@ -1037,11 +1044,14 @@ public Statistics visitPhysicalCTEProducer(PhysicalCTEProducer c Void context) { Statistics statistics = groupExpression.childStatistics(0); cteIdToStats.put(cteProducer.getCteId(), statistics); + cascadesContext.updateConsumerStats(cteProducer.getCteId(), statistics); return statistics; } @Override public Statistics visitPhysicalCTEConsumer(PhysicalCTEConsumer cteConsumer, Void context) { + cascadesContext.addCTEConsumerGroup(cteConsumer.getCteId(), groupExpression.getOwnerGroup(), + cteConsumer.getProducerToConsumerSlotMap()); CTEId cteId = cteConsumer.getCteId(); Statistics prodStats = cteIdToStats.get(cteId); if (prodStats == null) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java index 63365901a8ec01..3c8be1594bd445 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java @@ -144,14 +144,14 @@ public void testFilter() { GroupExpression groupExpression = new GroupExpression(logicalFilter, ImmutableList.of(childGroup)); Group ownerGroup = newGroup(); groupExpression.setOwnerGroup(ownerGroup); - StatsCalculator.estimate(groupExpression); + StatsCalculator.estimate(groupExpression, null); Assertions.assertEquals((10000 * 0.1 * 0.05), ownerGroup.getStatistics().getRowCount(), 0.001); LogicalFilter logicalFilterOr = new LogicalFilter<>(or, groupPlan); GroupExpression groupExpressionOr = new GroupExpression(logicalFilterOr, ImmutableList.of(childGroup)); Group ownerGroupOr = newGroup(); groupExpressionOr.setOwnerGroup(ownerGroupOr); - StatsCalculator.estimate(groupExpressionOr); + StatsCalculator.estimate(groupExpressionOr, null); Assertions.assertEquals((long) (10000 * (0.1 + 0.05 - 0.1 * 0.05)), ownerGroupOr.getStatistics().getRowCount(), 0.001); } @@ -197,14 +197,14 @@ public void testFilterOutofRange() { GroupExpression groupExpression = new GroupExpression(logicalFilter, ImmutableList.of(childGroup)); Group ownerGroup = newGroup(); groupExpression.setOwnerGroup(ownerGroup); - StatsCalculator.estimate(groupExpression); + StatsCalculator.estimate(groupExpression, null); Assertions.assertEquals(0, ownerGroup.getStatistics().getRowCount(), 0.001); LogicalFilter logicalFilterOr = new LogicalFilter<>(or, groupPlan); GroupExpression groupExpressionOr = new GroupExpression(logicalFilterOr, ImmutableList.of(childGroup)); Group ownerGroupOr = newGroup(); groupExpressionOr.setOwnerGroup(ownerGroupOr); - StatsCalculator.estimate(groupExpressionOr); + StatsCalculator.estimate(groupExpressionOr, null); Assertions.assertEquals(0, ownerGroupOr.getStatistics().getRowCount(), 0.001); } // TODO: temporary disable this test, until we could get column stats @@ -259,7 +259,7 @@ public void testOlapScan(@Mocked ConnectContext context) { GroupExpression groupExpression = new GroupExpression(logicalOlapScan1, ImmutableList.of(childGroup)); Group ownerGroup = newGroup(); groupExpression.setOwnerGroup(ownerGroup); - StatsCalculator.estimate(groupExpression); + StatsCalculator.estimate(groupExpression, null); Statistics stats = ownerGroup.getStatistics(); Assertions.assertEquals(1, stats.columnStatistics().size()); Assertions.assertNotNull(stats.columnStatistics().get(slot1)); @@ -289,7 +289,7 @@ public void testLimit() { GroupExpression groupExpression = new GroupExpression(logicalLimit, ImmutableList.of(childGroup)); Group ownerGroup = newGroup(); ownerGroup.addGroupExpression(groupExpression); - StatsCalculator.estimate(groupExpression); + StatsCalculator.estimate(groupExpression, null); Statistics limitStats = ownerGroup.getStatistics(); Assertions.assertEquals(1, limitStats.getRowCount()); ColumnStatistic slot1Stats = limitStats.columnStatistics().get(slot1); @@ -319,7 +319,7 @@ public void testTopN() { GroupExpression groupExpression = new GroupExpression(logicalTopN, ImmutableList.of(childGroup)); Group ownerGroup = newGroup(); ownerGroup.addGroupExpression(groupExpression); - StatsCalculator.estimate(groupExpression); + StatsCalculator.estimate(groupExpression, null); Statistics topNStats = ownerGroup.getStatistics(); Assertions.assertEquals(1, topNStats.getRowCount()); ColumnStatistic slot1Stats = topNStats.columnStatistics().get(slot1); From 2684fbdfdae43d3961f9768206628f964a1849ca Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Wed, 19 Jul 2023 11:41:41 +0800 Subject: [PATCH 5/8] [refactor](Nereids) refactor cte analyze, rewrite and reuse code (#21727) REFACTOR: 1. Generate CTEAnchor, CTEProducer, CTEConsumer when analyze. For example, statement `WITH cte1 AS (SELECT * FROM t) SELECT * FROM cte1`. Before this PR, we got analyzed plan like this: ``` logicalCTE(LogicalSubQueryAlias(cte1)) +-- logicalProject() +-- logicalCteConsumer() ``` we only have LogicalCteConsumer on the plan, but not LogicalCteProducer. This is not a valid plan, and should not as the final result of analyze. After this PR, we got analyzed plan like this: ``` logicalCteAnchor() |-- logicalCteProducer() +-- logicalProject() +-- logicalCteConsumer() ``` This is a valid plan with LogicalCteProducer and LogicalCteConsumer 2. Replace re-analyze unbound plan with deepCopy plan when do CTEInline Because we generate LogicalCteAnchor and LogicalCteProducer when analyze. So, we could not do re-analyze to gnerate CTE inline plan anymore. The another reason is, we reuse relation id between unbound and bound relation. So, if we do re-analyze on unresloved CTE plan, we will get two relation with same RelationId. This is wrong, because we use RelationId to distinguish two different relations. This PR implement two helper class to deep copy a new plan from CTEProducer. `LogicalPlanDeepCopier` and `ExpressionDeepCopier` 3. New rewrite framework to ensure do CTEInline in right way. Before this PR, we do CTEInline before apply any rewrite rule. But sometimes, some CteConsumer could be eliminated after rewrite. After this PR, we do CTEInline after the plans relaying on CTEProducer have been rewritten. So we could do CTEInline if some the count of CTEConsumer decrease under the threshold of CTEInline. 4. add relation id to all relation plan node 5. let all relation generated from table implement trait CatalogRelation 6. reuse relation id between unbound relation and relation after bind ENHANCEMENT: 1. Pull up CTEAnchor before RBO to avoid break other rules' pattern Before this PR, we will generate CTEAnchor and LogicalCTE in the middle of plan. So all rules should process LogicalCTEAnchor, otherwise will generate unexpected plan. For example, push down filter and push down project should add pattern like: ``` logicalProject(logicalCTE) ... logicalFilter(logicalCteAnchor) ... ``` project and filter must be push through these virtual plan node to ensure all project and filter could be merged togather and get right order of them. for Example: ``` logicalProject +-- logicalFilter +-- logicalCteAnchor +-- logicalProject +-- logicalFilter +-- logicalOlapScan ``` upper plan will lead to translation error. because we could not do twice filter and project on bottom logicalOlapScan. BUGFIX: 1. Recursive analyze LogicalCTE to avoid bind outer relation on inner CTE For example ```sql SELECT * FROM (WITH cte1 AS (SELECT * FROM t1) SELECT * FROM cte1)v1, cte1 v2; ``` Before this PR, we will use nested cte name to bind outer plan. So the outer cte1 with alias v2 will bound on the inner cte1. After this PR, the sql will throw Table not exists exception when binding. 2. Use right way do withChildren in CTEProducer and remove projects in it Before this PR, we add an attr named projects in CTEProducer to represent the output of it. This is because we cannot get right output of it by call `getOutput` method on it. The root reason of that is the wrong implementation of computeOutput of LogicalCteProducer. This PR fix this problem and remove projects attr of CTEProducer. 3. Adjust nullable rule update CTEConsumer's output by CTEProducer's output This PR process nullable on LogicalCteConsumer to ensure CteConsumer's output with right nullable info, if the CteProducer's output nullable has been adjusted. 4. Bind set operation expression should not change children's output's nullable This PR use fix a problem introduced by prvious PR #21168. The nullable info of SetOperation's children should not changed after binding SetOperation. --- .../doris/catalog/FunctionGenTable.java | 2 +- .../org/apache/doris/nereids/CTEContext.java | 8 +- .../apache/doris/nereids/CascadesContext.java | 236 ++++------ .../apache/doris/nereids/NereidsPlanner.java | 4 +- .../doris/nereids/StatementContext.java | 79 +++- .../analyzer/UnboundOneRowRelation.java | 48 +- .../nereids/analyzer/UnboundRelation.java | 49 +- .../nereids/analyzer/UnboundTVFRelation.java | 42 +- .../translator/PhysicalPlanTranslator.java | 35 +- .../translator/RuntimeFilterTranslator.java | 4 +- .../doris/nereids/jobs/executor/Analyzer.java | 22 +- .../doris/nereids/jobs/executor/Rewriter.java | 87 +++- .../jobs/rewrite/CostBasedRewriteJob.java | 57 ++- .../rewrite/PlanTreeRewriteBottomUpJob.java | 11 +- .../rewrite/PlanTreeRewriteTopDownJob.java | 6 +- .../jobs/rewrite/RootPlanTreeRewriteJob.java | 3 - .../org/apache/doris/nereids/memo/Memo.java | 14 - .../nereids/parser/LogicalPlanBuilder.java | 13 +- .../processor/post/RuntimeFilterContext.java | 8 +- .../post/RuntimeFilterGenerator.java | 8 +- .../processor/post/RuntimeFilterPruner.java | 2 +- .../apache/doris/nereids/rules/RuleSet.java | 16 +- .../apache/doris/nereids/rules/RuleType.java | 15 +- .../nereids/rules/analysis/AnalyzeCTE.java | 127 ++++++ .../rules/analysis/BindExpression.java | 28 +- .../nereids/rules/analysis/BindRelation.java | 45 +- .../rules/analysis/CheckAfterRewrite.java | 10 +- .../rules/analysis/ColumnAliasGenerator.java | 7 +- .../LogicalSubQueryAliasToLogicalProject.java | 3 +- .../nereids/rules/analysis/RegisterCTE.java | 127 ------ .../nereids/rules/analysis/SlotBinder.java | 15 +- .../rules/analysis/SubExprAnalyzer.java | 4 +- .../rules/analysis/UserAuthentication.java | 8 +- .../rules/expression/ExpressionRewrite.java | 2 +- ...icalCTEConsumerToPhysicalCTEConsumer.java} | 15 +- ...icalCTEProducerToPhysicalCTEProducer.java} | 7 +- ...lEmptyRelationToPhysicalEmptyRelation.java | 3 +- .../LogicalEsScanToPhysicalEsScan.java | 2 +- .../LogicalFileScanToPhysicalFileScan.java | 2 +- .../LogicalJdbcScanToPhysicalJdbcScan.java | 4 +- .../LogicalOlapScanToPhysicalOlapScan.java | 2 +- ...neRowRelationToPhysicalOneRowRelation.java | 2 +- ...LogicalSchemaScanToPhysicalSchemaScan.java | 2 +- ...gicalTVFRelationToPhysicalTVFRelation.java | 2 +- .../AddDefaultLimit.java | 38 +- .../nereids/rules/rewrite/AdjustNullable.java | 148 +++--- .../AggScalarSubQueryToWindowFunction.java | 16 +- .../rewrite/BuildCTEAnchorAndCTEProducer.java | 67 --- .../nereids/rules/rewrite/CTEInline.java | 112 +++++ .../rules/rewrite/CTEProducerRewrite.java | 122 ----- .../rules/rewrite/CheckMatchExpression.java | 7 +- .../rewrite/CollectFilterAboveConsumer.java | 4 +- .../rewrite/CollectProjectAboveConsumer.java | 6 +- .../nereids/rules/rewrite/ColumnPruning.java | 8 +- .../rules/rewrite/EliminateFilter.java | 7 +- .../nereids/rules/rewrite/EliminateLimit.java | 3 +- .../rewrite/EliminateUnnecessaryProject.java | 3 +- .../nereids/rules/rewrite/InApplyToJoin.java | 5 +- .../nereids/rules/rewrite/InlineCTE.java | 73 --- .../rules/rewrite/PullUpCteAnchor.java | 90 ++++ .../PushdownFilterThroughAggregation.java | 7 +- .../rewrite/PushdownFilterThroughCTE.java | 39 -- .../PushdownFilterThroughCTEAnchor.java | 39 -- .../rewrite/PushdownFilterThroughWindow.java | 6 +- .../nereids/rules/rewrite/PushdownLimit.java | 4 +- .../rewrite/PushdownProjectThroughCTE.java | 39 -- .../PushdownProjectThroughCTEAnchor.java | 39 -- .../nereids/rules/rewrite/ReorderJoin.java | 2 +- .../rules/rewrite/RewriteCteChildren.java | 189 ++++++++ .../doris/nereids/stats/StatsCalculator.java | 36 +- .../trees/copier/DeepCopierContext.java | 50 ++ .../trees/copier/ExpressionDeepCopier.java | 122 +++++ .../trees/copier/LogicalPlanDeepCopier.java | 429 ++++++++++++++++++ .../nereids/trees/expressions/Exists.java | 5 +- .../expressions/MarkJoinSlotReference.java | 12 + .../doris/nereids/trees/expressions/Slot.java | 6 +- .../trees/expressions/SlotReference.java | 28 +- .../StatementScopeIdGenerator.java | 9 + .../trees/expressions/SubqueryExpr.java | 4 + .../expressions/VirtualSlotReference.java | 26 ++ .../functions/table/TableValuedFunction.java | 3 +- .../doris/nereids/trees/plans/ObjectId.java | 2 +- .../doris/nereids/trees/plans/Plan.java | 2 +- .../doris/nereids/trees/plans/RelationId.java | 68 +++ .../trees/plans/algebra/CatalogRelation.java | 5 +- .../nereids/trees/plans/algebra/OlapScan.java | 11 +- .../nereids/trees/plans/algebra/Relation.java | 4 + .../nereids/trees/plans/algebra/Scan.java | 27 -- .../plans/logical/AbstractLogicalPlan.java | 2 +- .../trees/plans/logical/LogicalAggregate.java | 10 +- .../trees/plans/logical/LogicalCTE.java | 56 +-- .../trees/plans/logical/LogicalCTEAnchor.java | 16 +- .../plans/logical/LogicalCTEConsumer.java | 117 +++-- .../plans/logical/LogicalCTEProducer.java | 69 +-- .../plans/logical/LogicalCatalogRelation.java | 98 ++++ .../plans/logical/LogicalEmptyRelation.java | 46 +- .../trees/plans/logical/LogicalEsScan.java | 15 +- .../trees/plans/logical/LogicalFileScan.java | 36 +- .../trees/plans/logical/LogicalFilter.java | 2 +- .../trees/plans/logical/LogicalGenerate.java | 11 +- .../trees/plans/logical/LogicalJdbcScan.java | 15 +- .../trees/plans/logical/LogicalOlapScan.java | 89 ++-- .../plans/logical/LogicalOneRowRelation.java | 36 +- .../plans/logical/LogicalPartitionTopN.java | 5 + .../trees/plans/logical/LogicalRelation.java | 80 +--- .../plans/logical/LogicalSchemaScan.java | 24 +- .../plans/logical/LogicalSetOperation.java | 41 +- .../plans/logical/LogicalSubQueryAlias.java | 39 +- .../plans/logical/LogicalTVFRelation.java | 49 +- .../plans/physical/PhysicalCTEAnchor.java | 21 +- .../plans/physical/PhysicalCTEConsumer.java | 109 ++--- .../plans/physical/PhysicalCTEProducer.java | 38 +- .../physical/PhysicalCatalogRelation.java | 116 +++++ .../plans/physical/PhysicalEmptyRelation.java | 47 +- .../trees/plans/physical/PhysicalEsScan.java | 54 +-- .../plans/physical/PhysicalFileScan.java | 53 +-- .../plans/physical/PhysicalJdbcScan.java | 32 +- .../plans/physical/PhysicalOlapScan.java | 53 +-- .../physical/PhysicalOneRowRelation.java | 39 +- .../plans/physical/PhysicalRelation.java | 31 +- .../plans/physical/PhysicalSchemaScan.java | 47 +- .../PhysicalStorageLayerAggregate.java | 48 +- .../plans/physical/PhysicalTVFRelation.java | 54 ++- .../doris/nereids/util/RelationUtil.java | 25 - .../org/apache/doris/qe/SessionVariable.java | 12 +- .../nereids/datasets/tpch/TPCHTestBase.java | 4 + .../PhysicalPlanTranslatorTest.java | 4 +- .../nereids/jobs/RewriteTopDownJobTest.java | 12 +- .../jobs/cascades/DeriveStatsJobTest.java | 4 +- .../apache/doris/nereids/memo/MemoTest.java | 184 ++++---- .../pattern/GroupExpressionMatchingTest.java | 36 +- .../MergeProjectPostProcessTest.java | 4 +- .../PushdownFilterThroughProjectTest.java | 4 +- ...gisterCTETest.java => AnalyzeCTETest.java} | 254 +++++------ .../rules/analysis/BindRelationTest.java | 6 +- .../rules/analysis/BindSlotReferenceTest.java | 15 +- .../rules/analysis/CheckAnalysisTest.java | 5 +- .../rules/analysis/CheckRowPolicyTest.java | 8 +- .../ExpressionRewriteTestHelper.java | 4 +- .../rules/expression/FoldConstantTest.java | 6 +- .../rules/expression/SimplifyRangeTest.java | 4 +- .../LogicalWindowToPhysicalWindowTest.java | 4 +- ...AggScalarSubQueryToWindowFunctionTest.java | 4 +- .../rewrite/AggregateStrategiesTest.java | 4 +- ...CheckAndStandardizeWindowFunctionTest.java | 4 +- .../rewrite/EliminateGroupByConstantTest.java | 12 +- .../EliminateUnnecessaryProjectTest.java | 16 +- ...tractAndNormalizeWindowExpressionTest.java | 4 +- .../rules/rewrite/MergeProjectsTest.java | 4 +- .../rules/rewrite/NormalizeAggregateTest.java | 4 +- .../rewrite/PruneOlapScanTabletTest.java | 4 +- .../PushdownFilterThroughAggregationTest.java | 4 +- .../PushdownFilterThroughSortTest.java | 4 +- .../PushdownFilterThroughWindowTest.java | 4 +- .../rules/rewrite/PushdownLimitTest.java | 10 +- .../nereids/stats/StatsCalculatorTest.java | 4 +- .../nereids/trees/plans/PlanEqualsTest.java | 24 +- .../nereids/trees/plans/PlanOutputTest.java | 4 +- .../doris/nereids/util/MemoTestUtils.java | 2 +- .../doris/nereids/util/PlanChecker.java | 57 ++- .../doris/nereids/util/PlanConstructor.java | 14 +- .../doris/nereids/util/PlanRewriter.java | 54 +-- .../doris/utframe/TestWithFeService.java | 10 + .../shape/query1.out | 8 +- .../shape/query11.out | 12 +- .../shape/query14.out | 55 +-- .../shape/query2.out | 8 +- .../shape/query23.out | 16 +- .../shape/query24.out | 8 +- .../shape/query30.out | 8 +- .../shape/query31.out | 20 +- .../shape/query39.out | 8 +- .../shape/query4.out | 16 +- .../shape/query47.out | 10 +- .../shape/query54.out | 4 +- .../shape/query57.out | 10 +- .../shape/query59.out | 32 +- .../shape/query74.out | 16 +- .../shape/query75.out | 8 +- .../shape/query81.out | 8 +- .../shape/query95.out | 8 +- 181 files changed, 3143 insertions(+), 2660 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/RegisterCTE.java rename fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/{LogicalCTEConsumeToPhysicalCTEConsume.java => LogicalCTEConsumerToPhysicalCTEConsumer.java} (73%) rename fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/{LogicalCTEProduceToPhysicalCTEProduce.java => LogicalCTEProducerToPhysicalCTEProducer.java} (86%) rename fe/fe-core/src/main/java/org/apache/doris/nereids/rules/{analysis => rewrite}/AddDefaultLimit.java (72%) delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/BuildCTEAnchorAndCTEProducer.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEInline.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEProducerRewrite.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InlineCTE.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughCTE.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughCTEAnchor.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownProjectThroughCTE.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownProjectThroughCTEAnchor.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/RewriteCteChildren.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/DeepCopierContext.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/ExpressionDeepCopier.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/RelationId.java delete mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Scan.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCatalogRelation.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCatalogRelation.java rename fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/{RegisterCTETest.java => AnalyzeCTETest.java} (57%) diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionGenTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionGenTable.java index c4181fa991bea1..748564e9ace159 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionGenTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/FunctionGenTable.java @@ -22,7 +22,7 @@ import java.util.List; public class FunctionGenTable extends Table { - private TableValuedFunctionIf tvf; + private final TableValuedFunctionIf tvf; public FunctionGenTable(long id, String tableName, TableType type, List fullSchema, TableValuedFunctionIf tvf) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CTEContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CTEContext.java index d00684a70ec676..6bc67870708e00 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CTEContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CTEContext.java @@ -43,14 +43,14 @@ public class CTEContext { /* build head CTEContext */ public CTEContext() { - this(null, null, CTEId.DEFAULT); + this(CTEId.DEFAULT, null, null); } /** * CTEContext */ - public CTEContext(@Nullable LogicalSubQueryAlias parsedPlan, - @Nullable CTEContext previousCteContext, CTEId cteId) { + public CTEContext(CTEId cteId, @Nullable LogicalSubQueryAlias parsedPlan, + @Nullable CTEContext previousCteContext) { if ((parsedPlan == null && previousCteContext != null) || (parsedPlan != null && previousCteContext == null)) { throw new AnalysisException("Only first CteContext can contains null cte plan or previousCteContext"); } @@ -78,7 +78,7 @@ public boolean containsCTE(String cteName) { /** * Get for CTE reuse. */ - public Optional getReuse(String cteName) { + public Optional getAnalyzedCTEPlan(String cteName) { if (!findCTEContext(cteName).isPresent()) { return Optional.empty(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index 2900cc0d8898f5..dab6377addf0f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -28,10 +28,8 @@ import org.apache.doris.nereids.jobs.Job; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.jobs.executor.Analyzer; -import org.apache.doris.nereids.jobs.rewrite.CustomRewriteJob; import org.apache.doris.nereids.jobs.rewrite.RewriteBottomUpJob; import org.apache.doris.nereids.jobs.rewrite.RewriteTopDownJob; -import org.apache.doris.nereids.jobs.rewrite.RootPlanTreeRewriteJob.RootRewriteJobContext; import org.apache.doris.nereids.jobs.scheduler.JobPool; import org.apache.doris.nereids.jobs.scheduler.JobScheduler; import org.apache.doris.nereids.jobs.scheduler.JobStack; @@ -41,22 +39,21 @@ import org.apache.doris.nereids.memo.Memo; import org.apache.doris.nereids.processor.post.RuntimeFilterContext; import org.apache.doris.nereids.properties.PhysicalProperties; -import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleFactory; import org.apache.doris.nereids.rules.RuleSet; -import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SubqueryExpr; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; -import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; import org.apache.doris.statistics.ColumnStatistic; @@ -71,10 +68,10 @@ import java.util.List; import java.util.Map; import java.util.Map.Entry; +import java.util.Objects; import java.util.Optional; import java.util.Set; import java.util.Stack; -import java.util.concurrent.Callable; import java.util.concurrent.TimeUnit; import java.util.function.Function; import javax.annotation.Nullable; @@ -86,12 +83,11 @@ public class CascadesContext implements ScheduleContext { // in analyze/rewrite stage, the plan will storage in this field private Plan plan; - private Optional currentRootRewriteJobContext; // in optimize stage, the plan will storage in the memo private Memo memo; private final StatementContext statementContext; - private CTEContext cteContext; + private final CTEContext cteContext; private final RuleSet ruleSet; private final JobPool jobPool; private final JobScheduler jobScheduler; @@ -105,19 +101,9 @@ public class CascadesContext implements ScheduleContext { private boolean isRewriteRoot; private volatile boolean isTimeout = false; - private Map> cteIdToConsumers = new HashMap<>(); - private Map> cteIdToCTEClosure = new HashMap<>(); - private Map> cteIdToProjects = new HashMap<>(); - private Map> consumerIdToFilters = new HashMap<>(); - private Map> cteIdToConsumerUnderProjects = new HashMap<>(); - - // Used to update consumer's stats - private Map, Group>>> cteIdToConsumerGroup = new HashMap<>(); - - public CascadesContext(Plan plan, Memo memo, StatementContext statementContext, - PhysicalProperties requestProperties) { - this(plan, memo, statementContext, new CTEContext(), requestProperties); - } + // current process subtree, represent outer plan if empty + private final Optional currentTree; + private final Optional parent; /** * Constructor of OptimizerContext. @@ -125,55 +111,76 @@ public CascadesContext(Plan plan, Memo memo, StatementContext statementContext, * @param memo {@link Memo} reference * @param statementContext {@link StatementContext} reference */ - public CascadesContext(Plan plan, Memo memo, StatementContext statementContext, + private CascadesContext(Optional parent, Optional currentTree, + StatementContext statementContext, Plan plan, Memo memo, CTEContext cteContext, PhysicalProperties requireProperties) { - this.plan = plan; + this.parent = Objects.requireNonNull(parent, "parent should not null"); + this.currentTree = Objects.requireNonNull(currentTree, "currentTree should not null"); + this.statementContext = Objects.requireNonNull(statementContext, "statementContext should not null"); + this.plan = Objects.requireNonNull(plan, "plan should not null"); this.memo = memo; - this.statementContext = statementContext; + this.cteContext = Objects.requireNonNull(cteContext, "cteContext should not null"); this.ruleSet = new RuleSet(); this.jobPool = new JobStack(); this.jobScheduler = new SimpleJobScheduler(); this.currentJobContext = new JobContext(this, requireProperties, Double.MAX_VALUE); this.subqueryExprIsAnalyzed = new HashMap<>(); this.runtimeFilterContext = new RuntimeFilterContext(getConnectContext().getSessionVariable()); - this.cteContext = cteContext; } - public static CascadesContext newRewriteContext(StatementContext statementContext, + /** + * init a brand-new context to process whole tree + */ + public static CascadesContext initContext(StatementContext statementContext, Plan initPlan, PhysicalProperties requireProperties) { - return new CascadesContext(initPlan, null, statementContext, requireProperties); + return newContext(Optional.empty(), Optional.empty(), statementContext, + initPlan, new CTEContext(), requireProperties); } - public static CascadesContext newRewriteContext(StatementContext statementContext, + /** + * use for analyze cte. we must pass CteContext from outer since we need to get right scope of cte + */ + public static CascadesContext newContextWithCteContext(CascadesContext cascadesContext, Plan initPlan, CTEContext cteContext) { - return newRewriteContext(statementContext, initPlan, cteContext, PhysicalProperties.ANY); + return newContext(Optional.of(cascadesContext), Optional.empty(), + cascadesContext.getStatementContext(), initPlan, cteContext, PhysicalProperties.ANY); } - public static CascadesContext newRewriteContext(StatementContext statementContext, - Plan initPlan, CTEContext cteContext, PhysicalProperties requireProperties) { - return new CascadesContext(initPlan, null, statementContext, cteContext, requireProperties); - } - - /** - * New rewrite context. - */ - public static CascadesContext newRewriteContext(CascadesContext context, Plan plan) { - return newRewriteContext(context, plan, PhysicalProperties.ANY); + public static CascadesContext newCurrentTreeContext(CascadesContext context) { + return CascadesContext.newContext(context.getParent(), context.getCurrentTree(), context.getStatementContext(), + context.getRewritePlan(), context.getCteContext(), + context.getCurrentJobContext().getRequiredProperties()); } /** * New rewrite context copy from current context, used in cbo rewriter. */ - public static CascadesContext newRewriteContext(CascadesContext context, + public static CascadesContext newSubtreeContext(Optional subtree, CascadesContext context, Plan plan, PhysicalProperties requireProperties) { - CascadesContext cascadesContext = CascadesContext.newRewriteContext( - context.getStatementContext(), plan, context.getCteContext(), requireProperties); - cascadesContext.cteIdToConsumers = context.cteIdToConsumers; - cascadesContext.cteIdToProjects = context.cteIdToProjects; - cascadesContext.cteContext = context.cteContext; - cascadesContext.cteIdToCTEClosure = context.cteIdToCTEClosure; - cascadesContext.consumerIdToFilters = context.consumerIdToFilters; - return cascadesContext; + return CascadesContext.newContext(Optional.of(context), subtree, context.getStatementContext(), + plan, context.getCteContext(), requireProperties); + } + + private static CascadesContext newContext(Optional parent, Optional subtree, + StatementContext statementContext, Plan initPlan, + CTEContext cteContext, PhysicalProperties requireProperties) { + return new CascadesContext(parent, subtree, statementContext, initPlan, null, cteContext, requireProperties); + } + + public CascadesContext getRoot() { + CascadesContext root = this; + while (root.getParent().isPresent()) { + root = root.getParent().get(); + } + return root; + } + + public Optional getParent() { + return parent; + } + + public Optional getCurrentTree() { + return currentTree; } public synchronized void setIsTimeout(boolean isTimeout) { @@ -196,10 +203,6 @@ public Analyzer newAnalyzer(Optional customTableResolver) { return new Analyzer(this, customTableResolver); } - public Analyzer newCustomAnalyzer(Optional customTableResolver) { - return new Analyzer(this, customTableResolver); - } - @Override public void pushJob(Job job) { jobPool.push(job); @@ -259,15 +262,6 @@ public void setRewritePlan(Plan plan) { this.plan = plan; } - public Optional getCurrentRootRewriteJobContext() { - return currentRootRewriteJobContext; - } - - public void setCurrentRootRewriteJobContext( - RootRewriteJobContext currentRootRewriteJobContext) { - this.currentRootRewriteJobContext = Optional.ofNullable(currentRootRewriteJobContext); - } - public void setSubqueryExprIsAnalyzed(SubqueryExpr subqueryExpr, boolean isAnalyzed) { subqueryExprIsAnalyzed.put(subqueryExpr, isAnalyzed); } @@ -284,41 +278,14 @@ public CascadesContext bottomUpRewrite(RuleFactory... rules) { return execute(new RewriteBottomUpJob(memo.getRoot(), currentJobContext, ImmutableList.copyOf(rules))); } - public CascadesContext bottomUpRewrite(Rule... rules) { - return bottomUpRewrite(ImmutableList.copyOf(rules)); - } - - public CascadesContext bottomUpRewrite(List rules) { - return execute(new RewriteBottomUpJob(memo.getRoot(), rules, currentJobContext)); - } - public CascadesContext topDownRewrite(RuleFactory... rules) { return execute(new RewriteTopDownJob(memo.getRoot(), currentJobContext, ImmutableList.copyOf(rules))); } - public CascadesContext topDownRewrite(Rule... rules) { - return topDownRewrite(ImmutableList.copyOf(rules)); - } - - public CascadesContext topDownRewrite(List rules) { - return execute(new RewriteTopDownJob(memo.getRoot(), rules, currentJobContext)); - } - - public CascadesContext topDownRewrite(CustomRewriter customRewriter) { - CustomRewriteJob customRewriteJob = new CustomRewriteJob(() -> customRewriter, RuleType.TEST_REWRITE); - customRewriteJob.execute(currentJobContext); - toMemo(); - return this; - } - public CTEContext getCteContext() { return cteContext; } - public void setCteContext(CTEContext cteContext) { - this.cteContext = cteContext; - } - public void setIsRewriteRoot(boolean isRewriteRoot) { this.isRewriteRoot = isRewriteRoot; } @@ -349,9 +316,8 @@ public T getAndCacheSessionVariable(String cacheName, if (statementContext == null) { return defaultValue; } - T cacheResult = statementContext.getOrRegisterCache(cacheName, + return statementContext.getOrRegisterCache(cacheName, () -> variableSupplier.apply(connectContext.getSessionVariable())); - return cacheResult; } private CascadesContext execute(Job job) { @@ -394,9 +360,9 @@ private Set getTables(LogicalPlan logicalPlan) { Set unboundRelations = new HashSet<>(); logicalPlan.foreach(p -> { if (p instanceof LogicalFilter) { - unboundRelations.addAll(extractUnboundRelationFromFilter((LogicalFilter) p)); + unboundRelations.addAll(extractUnboundRelationFromFilter((LogicalFilter) p)); } else if (p instanceof LogicalCTE) { - unboundRelations.addAll(extractUnboundRelationFromCTE((LogicalCTE) p)); + unboundRelations.addAll(extractUnboundRelationFromCTE((LogicalCTE) p)); } else { unboundRelations.addAll(p.collect(UnboundRelation.class::isInstance)); } @@ -404,7 +370,7 @@ private Set getTables(LogicalPlan logicalPlan) { return unboundRelations; } - private Set extractUnboundRelationFromFilter(LogicalFilter filter) { + private Set extractUnboundRelationFromFilter(LogicalFilter filter) { Set subqueryExprs = filter.getPredicate() .collect(SubqueryExpr.class::isInstance); Set relations = new HashSet<>(); @@ -415,7 +381,7 @@ private Set extractUnboundRelationFromFilter(LogicalFilter filt return relations; } - private Set extractUnboundRelationFromCTE(LogicalCTE cte) { + private Set extractUnboundRelationFromCTE(LogicalCTE cte) { List> subQueryAliases = cte.getAliasQueries(); Set relations = new HashSet<>(); for (LogicalSubQueryAlias subQueryAlias : subQueryAliases) { @@ -506,93 +472,49 @@ public void close() { } } - public void putCTEIdToCTEClosure(CTEId cteId, Callable cteClosure) { - this.cteIdToCTEClosure.put(cteId, cteClosure); - } - - public void putAllCTEIdToCTEClosure(Map> cteConsumers) { - this.cteIdToCTEClosure.putAll(cteConsumers); - } - public void putCTEIdToConsumer(LogicalCTEConsumer cteConsumer) { - Set consumers = - this.cteIdToConsumers.computeIfAbsent(cteConsumer.getCteId(), k -> new HashSet<>()); + Set consumers = this.statementContext.getCteIdToConsumers() + .computeIfAbsent(cteConsumer.getCteId(), k -> new HashSet<>()); consumers.add(cteConsumer); } - public void putAllCTEIdToConsumer(Map> cteConsumers) { - this.cteIdToConsumers.putAll(cteConsumers); - } - - public void putCTEIdToProject(CTEId cteId, Expression p) { - Set projects = this.cteIdToProjects.computeIfAbsent(cteId, k -> new HashSet<>()); + public void putCTEIdToProject(CTEId cteId, NamedExpression p) { + Set projects = this.statementContext.getCteIdToProjects() + .computeIfAbsent(cteId, k -> new HashSet<>()); projects.add(p); } - public Set getProjectForProducer(CTEId cteId) { - return this.cteIdToProjects.get(cteId); - } - - /** - * Fork for rewritten child tree of CTEProducer. - */ - public CascadesContext forkForCTEProducer(Plan plan) { - CascadesContext cascadesContext = new CascadesContext(plan, memo, statementContext, PhysicalProperties.ANY); - cascadesContext.cteIdToConsumers = cteIdToConsumers; - cascadesContext.cteIdToProjects = cteIdToProjects; - cascadesContext.cteContext = cteContext; - cascadesContext.cteIdToCTEClosure = cteIdToCTEClosure; - cascadesContext.consumerIdToFilters = consumerIdToFilters; - return cascadesContext; - } - - public int cteReferencedCount(CTEId cteId) { - Set cteConsumer = cteIdToConsumers.get(cteId); - if (cteConsumer == null) { - return 0; - } - return cteIdToConsumers.get(cteId).size(); + public Set getProjectForProducer(CTEId cteId) { + return this.statementContext.getCteIdToProjects().get(cteId); } public Map> getCteIdToConsumers() { - return cteIdToConsumers; - } - - public Map> getCteIdToCTEClosure() { - return cteIdToCTEClosure; - } - - public LogicalPlan findCTEPlanForInline(CTEId cteId) { - try { - return cteIdToCTEClosure.get(cteId).call(); - } catch (Exception e) { - throw new RuntimeException(e); - } + return this.statementContext.getCteIdToConsumers(); } - public void putConsumerIdToFilter(int id, Expression filter) { - Set filters = this.consumerIdToFilters.computeIfAbsent(id, k -> new HashSet<>()); + public void putConsumerIdToFilter(RelationId id, Expression filter) { + Set filters = this.getConsumerIdToFilters().computeIfAbsent(id, k -> new HashSet<>()); filters.add(filter); } - public Map> getConsumerIdToFilters() { - return consumerIdToFilters; + public Map> getConsumerIdToFilters() { + return this.statementContext.getConsumerIdToFilters(); } public void markConsumerUnderProject(LogicalCTEConsumer cteConsumer) { - Set consumerIds = - this.cteIdToConsumerUnderProjects.computeIfAbsent(cteConsumer.getCteId(), k -> new HashSet<>()); - consumerIds.add(cteConsumer.getConsumerId()); + Set consumerIds = this.statementContext.getCteIdToConsumerUnderProjects() + .computeIfAbsent(cteConsumer.getCteId(), k -> new HashSet<>()); + consumerIds.add(cteConsumer.getRelationId()); } public boolean couldPruneColumnOnProducer(CTEId cteId) { - Set consumerIds = this.cteIdToConsumerUnderProjects.get(cteId); - return consumerIds.size() == this.cteIdToConsumers.get(cteId).size(); + Set consumerIds = this.statementContext.getCteIdToConsumerUnderProjects().get(cteId); + return consumerIds.size() == this.statementContext.getCteIdToConsumers().get(cteId).size(); } public void addCTEConsumerGroup(CTEId cteId, Group g, Map producerSlotToConsumerSlot) { List, Group>> consumerGroups = - this.cteIdToConsumerGroup.computeIfAbsent(cteId, k -> new ArrayList<>()); + this.statementContext.getCteIdToConsumerGroup().computeIfAbsent(cteId, k -> new ArrayList<>()); consumerGroups.add(Pair.of(producerSlotToConsumerSlot, g)); } @@ -600,7 +522,7 @@ public void addCTEConsumerGroup(CTEId cteId, Group g, Map producerSl * Update CTE consumer group as producer's stats update */ public void updateConsumerStats(CTEId cteId, Statistics statistics) { - List, Group>> consumerGroups = this.cteIdToConsumerGroup.get(cteId); + List, Group>> consumerGroups = this.statementContext.getCteIdToConsumerGroup().get(cteId); for (Pair, Group> p : consumerGroups) { Map producerSlotToConsumerSlot = p.first; Statistics updatedConsumerStats = new Statistics(statistics); 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 25064c1c6824b6..02df5e9fa43b4f 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 @@ -268,7 +268,7 @@ private LogicalPlan preprocess(LogicalPlan logicalPlan) { } private void initCascadesContext(LogicalPlan plan, PhysicalProperties requireProperties) { - cascadesContext = CascadesContext.newRewriteContext(statementContext, plan, requireProperties); + cascadesContext = CascadesContext.initContext(statementContext, plan, requireProperties); if (statementContext.getConnectContext().getTables() != null) { cascadesContext.setTables(statementContext.getConnectContext().getTables()); } @@ -283,7 +283,7 @@ private void analyze() { * Logical plan rewrite based on a series of heuristic rules. */ private void rewrite() { - new Rewriter(cascadesContext).execute(); + Rewriter.getWholeTreeRewriter(cascadesContext).execute(); NereidsTracer.logImportantTime("EndRewritePlan"); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java index aaeefabf86cd78..b96aea672ec6fa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java @@ -19,10 +19,18 @@ import org.apache.doris.analysis.StatementBase; import org.apache.doris.common.IdGenerator; +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.rules.analysis.ColumnAliasGenerator; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; @@ -30,7 +38,8 @@ import com.google.common.base.Suppliers; import com.google.common.collect.Maps; -import java.util.HashSet; +import java.util.HashMap; +import java.util.List; import java.util.Map; import java.util.Set; import javax.annotation.concurrent.GuardedBy; @@ -42,30 +51,32 @@ public class StatementContext { private ConnectContext connectContext; + @GuardedBy("this") + private final Map> contextCacheMap = Maps.newLinkedHashMap(); + private OriginStatement originStatement; + // NOTICE: we set the plan parsed by DorisParser to parsedStatement and if the plan is command, create a + // LogicalPlanAdapter with the logical plan in the command. + private StatementBase parsedStatement; + private ColumnAliasGenerator columnAliasGenerator; private int joinCount = 0; private int maxNAryInnerJoin = 0; - private boolean isDpHyp = false; private boolean isOtherJoinReorder = false; private final IdGenerator exprIdGenerator = ExprId.createGenerator(); - private final IdGenerator objectIdGenerator = ObjectId.createGenerator(); - + private final IdGenerator relationIdGenerator = RelationId.createGenerator(); private final IdGenerator cteIdGenerator = CTEId.createGenerator(); - @GuardedBy("this") - private final Map> contextCacheMap = Maps.newLinkedHashMap(); - - // NOTICE: we set the plan parsed by DorisParser to parsedStatement and if the plan is command, create a - // LogicalPlanAdapter with the logical plan in the command. - private StatementBase parsedStatement; - - private Set columnNames; - - private ColumnAliasGenerator columnAliasGenerator; + private final Map> cteIdToConsumers = new HashMap<>(); + private final Map> cteIdToProjects = new HashMap<>(); + private final Map> consumerIdToFilters = new HashMap<>(); + private final Map> cteIdToConsumerUnderProjects = new HashMap<>(); + // Used to update consumer's stats + private final Map, Group>>> cteIdToConsumerGroup = new HashMap<>(); + private final Map rewrittenCtePlan = new HashMap<>(); public StatementContext() { this.connectContext = ConnectContext.get(); @@ -92,7 +103,7 @@ public OriginStatement getOriginStatement() { return originStatement; } - public void setMaxNArayInnerJoin(int maxNAryInnerJoin) { + public void setMaxNAryInnerJoin(int maxNAryInnerJoin) { if (maxNAryInnerJoin > this.maxNAryInnerJoin) { this.maxNAryInnerJoin = maxNAryInnerJoin; } @@ -140,6 +151,10 @@ public ObjectId getNextObjectId() { return objectIdGenerator.getNextId(); } + public RelationId getNextRelationId() { + return relationIdGenerator.getNextId(); + } + public void setParsedStatement(StatementBase parsedStatement) { this.parsedStatement = parsedStatement; } @@ -154,17 +169,9 @@ public synchronized T getOrRegisterCache(String key, Supplier cacheSuppli return supplier.get(); } - public Set getColumnNames() { - return columnNames == null ? new HashSet<>() : columnNames; - } - - public void setColumnNames(Set columnNames) { - this.columnNames = columnNames; - } - public ColumnAliasGenerator getColumnAliasGenerator() { return columnAliasGenerator == null - ? columnAliasGenerator = new ColumnAliasGenerator(this) + ? columnAliasGenerator = new ColumnAliasGenerator() : columnAliasGenerator; } @@ -175,4 +182,28 @@ public String generateColumnName() { public StatementBase getParsedStatement() { return parsedStatement; } + + public Map> getCteIdToConsumers() { + return cteIdToConsumers; + } + + public Map> getCteIdToProjects() { + return cteIdToProjects; + } + + public Map> getConsumerIdToFilters() { + return consumerIdToFilters; + } + + public Map> getCteIdToConsumerUnderProjects() { + return cteIdToConsumerUnderProjects; + } + + public Map, Group>>> getCteIdToConsumerGroup() { + return cteIdToConsumerGroup; + } + + public Map getRewrittenCtePlan() { + return rewrittenCtePlan; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java index 3f349c9224b9bc..dd04e9625a43cc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOneRowRelation.java @@ -24,11 +24,11 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.OneRowRelation; -import org.apache.doris.nereids.trees.plans.logical.LogicalLeaf; +import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -36,37 +36,30 @@ import com.google.common.collect.ImmutableList; import java.util.List; -import java.util.Objects; import java.util.Optional; /** * A relation that contains only one row consist of some constant expressions. * e.g. select 100, 'value' */ -public class UnboundOneRowRelation extends LogicalLeaf implements Unbound, OneRowRelation { +public class UnboundOneRowRelation extends LogicalRelation implements Unbound, OneRowRelation { - private final ObjectId id; private final List projects; - public UnboundOneRowRelation(ObjectId id, List projects) { - this(id, projects, Optional.empty(), Optional.empty()); + public UnboundOneRowRelation(RelationId relationId, List projects) { + this(relationId, projects, Optional.empty(), Optional.empty()); } - private UnboundOneRowRelation(ObjectId id, + private UnboundOneRowRelation(RelationId id, List projects, Optional groupExpression, Optional logicalProperties) { - super(PlanType.LOGICAL_UNBOUND_ONE_ROW_RELATION, groupExpression, logicalProperties); + super(id, PlanType.LOGICAL_UNBOUND_ONE_ROW_RELATION, groupExpression, logicalProperties); Preconditions.checkArgument(projects.stream().noneMatch(p -> p.containsType(Slot.class)), "OneRowRelation can not contains any slot"); - this.id = id; this.projects = ImmutableList.copyOf(projects); } - public ObjectId getId() { - return id; - } - @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitUnboundOneRowRelation(this, context); @@ -84,13 +77,14 @@ public List getExpressions() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new UnboundOneRowRelation(id, projects, groupExpression, Optional.of(logicalPropertiesSupplier.get())); + return new UnboundOneRowRelation(relationId, projects, + groupExpression, Optional.of(logicalPropertiesSupplier.get())); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new UnboundOneRowRelation(id, projects, groupExpression, logicalProperties); + return new UnboundOneRowRelation(relationId, projects, groupExpression, logicalProperties); } @Override @@ -106,28 +100,8 @@ public LogicalProperties computeLogicalProperties() { @Override public String toString() { return Utils.toSqlString("UnboundOneRowRelation", - "relationId", id, + "relationId", relationId, "projects", projects ); } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - UnboundOneRowRelation that = (UnboundOneRowRelation) o; - return Objects.equals(id, that.id) && Objects.equals(projects, that.projects); - } - - @Override - public int hashCode() { - return Objects.hash(id, projects); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java index 1f820398d98ebf..6d625f7068ec2a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundRelation.java @@ -17,16 +17,15 @@ package org.apache.doris.nereids.analyzer; -import org.apache.doris.catalog.Table; import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -50,20 +49,20 @@ public class UnboundRelation extends LogicalRelation implements Unbound { private final boolean isTempPart; private final List hints; - public UnboundRelation(ObjectId id, List nameParts) { + public UnboundRelation(RelationId id, List nameParts) { this(id, nameParts, Optional.empty(), Optional.empty(), ImmutableList.of(), false, ImmutableList.of()); } - public UnboundRelation(ObjectId id, List nameParts, List partNames, boolean isTempPart) { + public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart) { this(id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, ImmutableList.of()); } - public UnboundRelation(ObjectId id, List nameParts, List partNames, boolean isTempPart, + public UnboundRelation(RelationId id, List nameParts, List partNames, boolean isTempPart, List hints) { this(id, nameParts, Optional.empty(), Optional.empty(), partNames, isTempPart, hints); } - public UnboundRelation(ObjectId id, List nameParts, Optional groupExpression, + public UnboundRelation(RelationId id, List nameParts, Optional groupExpression, Optional logicalProperties, List partNames, boolean isTempPart, List hints) { super(id, PlanType.LOGICAL_UNBOUND_RELATION, groupExpression, logicalProperties); @@ -73,11 +72,6 @@ public UnboundRelation(ObjectId id, List nameParts, Optional getNameParts() { return nameParts; } @@ -94,14 +88,15 @@ public LogicalProperties computeLogicalProperties() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new UnboundRelation(id, nameParts, groupExpression, Optional.of(getLogicalProperties()), partNames, - isTempPart, hints); + return new UnboundRelation(relationId, nameParts, + groupExpression, Optional.of(getLogicalProperties()), + partNames, isTempPart, hints); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new UnboundRelation(id, nameParts, groupExpression, logicalProperties, partNames, + return new UnboundRelation(relationId, nameParts, groupExpression, logicalProperties, partNames, isTempPart, hints); } @@ -113,7 +108,7 @@ public List computeOutput() { @Override public String toString() { List args = Lists.newArrayList( - "id", id, + "id", relationId, "nameParts", StringUtils.join(nameParts, ".") ); if (CollectionUtils.isNotEmpty(hints)) { @@ -133,30 +128,6 @@ public List getExpressions() { throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); } - public ObjectId getId() { - return id; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - UnboundRelation that = (UnboundRelation) o; - return id.equals(that.id); - } - - @Override - public int hashCode() { - return Objects.hash(id); - } - public List getPartNames() { return partNames; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java index 2116c63c2952fd..83c9d4fa286584 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundTVFRelation.java @@ -25,11 +25,11 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.TVFProperties; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.TVFRelation; -import org.apache.doris.nereids.trees.plans.logical.LogicalLeaf; +import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -38,20 +38,18 @@ import java.util.Optional; /** UnboundTVFRelation */ -public class UnboundTVFRelation extends LogicalLeaf implements TVFRelation, Unbound { +public class UnboundTVFRelation extends LogicalRelation implements TVFRelation, Unbound { - private final ObjectId id; private final String functionName; private final TVFProperties properties; - public UnboundTVFRelation(ObjectId id, String functionName, TVFProperties properties) { + public UnboundTVFRelation(RelationId id, String functionName, TVFProperties properties) { this(id, functionName, properties, Optional.empty(), Optional.empty()); } - public UnboundTVFRelation(ObjectId id, String functionName, TVFProperties properties, + public UnboundTVFRelation(RelationId id, String functionName, TVFProperties properties, Optional groupExpression, Optional logicalProperties) { - super(PlanType.LOGICAL_UNBOUND_TVF_RELATION, groupExpression, logicalProperties); - this.id = id; + super(id, PlanType.LOGICAL_UNBOUND_TVF_RELATION, groupExpression, logicalProperties); this.functionName = Objects.requireNonNull(functionName, "functionName can not be null"); this.properties = Objects.requireNonNull(properties, "properties can not be null"); } @@ -64,10 +62,6 @@ public TVFProperties getProperties() { return properties; } - public ObjectId getId() { - return id; - } - @Override public TableValuedFunction getFunction() { throw new UnboundException("getFunction"); @@ -95,14 +89,14 @@ public List computeOutput() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new UnboundTVFRelation(id, functionName, properties, groupExpression, + return new UnboundTVFRelation(relationId, functionName, properties, groupExpression, Optional.of(getLogicalProperties())); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new UnboundTVFRelation(id, functionName, properties, groupExpression, logicalProperties); + return new UnboundTVFRelation(relationId, functionName, properties, groupExpression, logicalProperties); } @Override @@ -112,24 +106,4 @@ public String toString() { "arguments", properties ); } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - UnboundTVFRelation that = (UnboundTVFRelation) o; - return functionName.equals(that.functionName) && properties.equals(that.properties) && id.equals(that.id); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), functionName, properties, id); - } } 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 a4f172ea10a35d..5c8227ccedf9b3 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 @@ -420,7 +420,7 @@ public PlanFragment visitPhysicalFileScan(PhysicalFileScan fileScan, PlanTransla context.addScanNode(scanNode); ScanNode finalScanNode = scanNode; context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(fileScan.getId()).forEach( + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(fileScan.getRelationId()).forEach( expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, finalScanNode, context) ) ); @@ -462,7 +462,7 @@ public PlanFragment visitPhysicalEsScan(PhysicalEsScan esScan, PlanTranslatorCon Utils.execWithUncheckedException(esScanNode::init); context.addScanNode(esScanNode); context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(esScan.getId()).forEach( + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(esScan.getRelationId()).forEach( expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, esScanNode, context) ) ); @@ -485,7 +485,7 @@ public PlanFragment visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, PlanTransla Utils.execWithUncheckedException(jdbcScanNode::init); context.addScanNode(jdbcScanNode); context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(jdbcScan.getId()).forEach( + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(jdbcScan.getRelationId()).forEach( expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, jdbcScanNode, context) ) ); @@ -551,8 +551,9 @@ public PlanFragment visitPhysicalOlapScan(PhysicalOlapScan olapScan, PlanTransla // 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) + runtimeFilterTranslator -> runtimeFilterTranslator.getTargetOnScanNode(olapScan.getRelationId()) + .forEach(expr -> runtimeFilterTranslator.translateRuntimeFilterTarget( + expr, olapScanNode, context) ) ); // TODO: we need to remove all finalizeForNereids @@ -609,8 +610,8 @@ public PlanFragment visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, PlanT TupleDescriptor tupleDescriptor = generateTupleDesc(slots, table, context); SchemaScanNode scanNode = new SchemaScanNode(context.nextPlanNodeId(), tupleDescriptor); context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(schemaScan.getId()).forEach( - expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(schemaScan.getRelationId()) + .forEach(expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) ) ); scanNode.finalizeForNereids(); @@ -624,14 +625,14 @@ public PlanFragment visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, PlanT @Override public PlanFragment visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, PlanTranslatorContext context) { List slots = tvfRelation.getLogicalProperties().getOutput(); - TupleDescriptor tupleDescriptor = generateTupleDesc(slots, tvfRelation.getTable(), context); + TupleDescriptor tupleDescriptor = generateTupleDesc(slots, tvfRelation.getFunction().getTable(), context); TableValuedFunctionIf catalogFunction = tvfRelation.getFunction().getCatalogFunction(); ScanNode scanNode = catalogFunction.getScanNode(context.nextPlanNodeId(), tupleDescriptor); Utils.execWithUncheckedException(scanNode::init); context.getRuntimeTranslator().ifPresent( - runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(tvfRelation.getId()).forEach( - expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) + runtimeFilterGenerator -> runtimeFilterGenerator.getTargetOnScanNode(tvfRelation.getRelationId()) + .forEach(expr -> runtimeFilterGenerator.translateRuntimeFilterTarget(expr, scanNode, context) ) ); Utils.execWithUncheckedException(scanNode::finalizeForNereids); @@ -830,7 +831,7 @@ public PlanFragment visitPhysicalCTEConsumer(PhysicalCTEConsumer cteConsumer, multiCastDataSink.getDestinations().add(Lists.newArrayList()); // update expr to slot mapping - for (Slot producerSlot : cteProducer.getProjects()) { + for (Slot producerSlot : cteProducer.getOutput()) { Slot consumerSlot = cteConsumer.getProducerToConsumerSlotMap().get(producerSlot); SlotRef slotRef = context.findSlotRef(producerSlot.getExprId()); context.addExprIdSlotRefPair(consumerSlot.getExprId(), slotRef); @@ -844,21 +845,21 @@ public PlanFragment visitPhysicalCTEProducer(PhysicalCTEProducer PlanFragment child = cteProducer.child().accept(this, context); CTEId cteId = cteProducer.getCteId(); context.getPlanFragments().remove(child); - MultiCastPlanFragment cteProduce = new MultiCastPlanFragment(child); + MultiCastPlanFragment multiCastPlanFragment = new MultiCastPlanFragment(child); MultiCastDataSink multiCastDataSink = new MultiCastDataSink(); - cteProduce.setSink(multiCastDataSink); + multiCastPlanFragment.setSink(multiCastDataSink); - List outputs = cteProducer.getProjects().stream() + List outputs = cteProducer.getOutput().stream() .map(e -> ExpressionTranslator.translate(e, context)) .collect(Collectors.toList()); - cteProduce.setOutputExprs(outputs); - context.getCteProduceFragments().put(cteId, cteProduce); + multiCastPlanFragment.setOutputExprs(outputs); + context.getCteProduceFragments().put(cteId, multiCastPlanFragment); context.getCteProduceMap().put(cteId, cteProducer); if (context.getRuntimeTranslator().isPresent()) { context.getRuntimeTranslator().get().getContext().getCteProduceMap().put(cteId, cteProducer); } - context.getPlanFragments().add(cteProduce); + context.getPlanFragments().add(multiCastPlanFragment); return child; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java index bbb9997f2e083b..6980f05fa5d945 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/RuntimeFilterTranslator.java @@ -28,7 +28,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalJoin; import org.apache.doris.nereids.trees.plans.physical.RuntimeFilter; import org.apache.doris.planner.HashJoinNode; @@ -68,7 +68,7 @@ public RuntimeFilterContext getContext() { return context; } - public List getTargetOnScanNode(ObjectId id) { + public List getTargetOnScanNode(RelationId id) { return context.getTargetOnOlapScanNodeMap().getOrDefault(id, Collections.emptyList()); } 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 7f089d14b1801c..96afa6b24ad0b5 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 @@ -20,6 +20,7 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.jobs.rewrite.RewriteJob; import org.apache.doris.nereids.rules.analysis.AdjustAggregateNullableForEmptySet; +import org.apache.doris.nereids.rules.analysis.AnalyzeCTE; import org.apache.doris.nereids.rules.analysis.BindExpression; import org.apache.doris.nereids.rules.analysis.BindInsertTargetTable; import org.apache.doris.nereids.rules.analysis.BindRelation; @@ -31,7 +32,6 @@ import org.apache.doris.nereids.rules.analysis.NormalizeRepeat; import org.apache.doris.nereids.rules.analysis.ProjectToGlobalAggregate; import org.apache.doris.nereids.rules.analysis.ProjectWithDistinctToAggregate; -import org.apache.doris.nereids.rules.analysis.RegisterCTE; import org.apache.doris.nereids.rules.analysis.ReplaceExpressionByChildOutput; import org.apache.doris.nereids.rules.analysis.ResolveOrdinalInOrderByAndGroupBy; import org.apache.doris.nereids.rules.analysis.SubqueryToApply; @@ -49,9 +49,7 @@ public class Analyzer extends AbstractBatchJobExecutor { public static final List DEFAULT_ANALYZE_JOBS = buildAnalyzeJobs(Optional.empty()); - private Optional customTableResolver; - - private List jobs; + private final List jobs; /** * Execute the analysis job with scope. @@ -63,7 +61,7 @@ public Analyzer(CascadesContext cascadesContext) { public Analyzer(CascadesContext cascadesContext, Optional customTableResolver) { super(cascadesContext); - this.customTableResolver = Objects.requireNonNull(customTableResolver, "customTableResolver cannot be null"); + Objects.requireNonNull(customTableResolver, "customTableResolver cannot be null"); this.jobs = !customTableResolver.isPresent() ? DEFAULT_ANALYZE_JOBS : buildAnalyzeJobs(customTableResolver); } @@ -81,21 +79,15 @@ public void analyze() { private static List buildAnalyzeJobs(Optional customTableResolver) { return jobs( - topDown( - new RegisterCTE() - ), + topDown(new AnalyzeCTE()), bottomUp( - new BindRelation(customTableResolver.orElse(null)), + new BindRelation(customTableResolver), new CheckPolicy(), new UserAuthentication(), new BindExpression() ), - topDown( - new BindInsertTargetTable() - ), - bottomUp( - new CheckBound() - ), + topDown(new BindInsertTargetTable()), + bottomUp(new CheckBound()), bottomUp( new ProjectToGlobalAggregate(), // this rule check's the logicalProject node's isDistinct property 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 ce3d8b35db82ee..16fbb6979b733b 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 @@ -18,11 +18,11 @@ package org.apache.doris.nereids.jobs.executor; import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.jobs.rewrite.CostBasedRewriteJob; import org.apache.doris.nereids.jobs.rewrite.RewriteJob; import org.apache.doris.nereids.processor.pre.EliminateLogicalSelectHint; import org.apache.doris.nereids.rules.RuleSet; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.rules.analysis.AddDefaultLimit; import org.apache.doris.nereids.rules.analysis.AdjustAggregateNullableForEmptySet; import org.apache.doris.nereids.rules.analysis.AvgDistinctToSumDivCount; import org.apache.doris.nereids.rules.analysis.CheckAfterRewrite; @@ -31,12 +31,12 @@ import org.apache.doris.nereids.rules.expression.ExpressionNormalization; import org.apache.doris.nereids.rules.expression.ExpressionOptimization; import org.apache.doris.nereids.rules.expression.ExpressionRewrite; +import org.apache.doris.nereids.rules.rewrite.AddDefaultLimit; import org.apache.doris.nereids.rules.rewrite.AdjustConjunctsReturnType; import org.apache.doris.nereids.rules.rewrite.AdjustNullable; import org.apache.doris.nereids.rules.rewrite.AggScalarSubQueryToWindowFunction; import org.apache.doris.nereids.rules.rewrite.BuildAggForUnion; -import org.apache.doris.nereids.rules.rewrite.BuildCTEAnchorAndCTEProducer; -import org.apache.doris.nereids.rules.rewrite.CTEProducerRewrite; +import org.apache.doris.nereids.rules.rewrite.CTEInline; import org.apache.doris.nereids.rules.rewrite.CheckAndStandardizeWindowFunctionAndFrame; import org.apache.doris.nereids.rules.rewrite.CheckDataTypes; import org.apache.doris.nereids.rules.rewrite.CheckMatchExpression; @@ -65,6 +65,7 @@ import org.apache.doris.nereids.rules.rewrite.InferJoinNotNull; import org.apache.doris.nereids.rules.rewrite.InferPredicates; import org.apache.doris.nereids.rules.rewrite.InlineCTE; +import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct; import org.apache.doris.nereids.rules.rewrite.MergeFilters; import org.apache.doris.nereids.rules.rewrite.MergeOneRowRelationIntoUnion; import org.apache.doris.nereids.rules.rewrite.MergeProjects; @@ -75,6 +76,7 @@ import org.apache.doris.nereids.rules.rewrite.PruneOlapScanPartition; import org.apache.doris.nereids.rules.rewrite.PruneOlapScanTablet; import org.apache.doris.nereids.rules.rewrite.PushConjunctsIntoJdbcScan; +import org.apache.doris.nereids.rules.rewrite.PullUpCteAnchor; import org.apache.doris.nereids.rules.rewrite.PushFilterInsideJoin; import org.apache.doris.nereids.rules.rewrite.PushProjectIntoOneRowRelation; import org.apache.doris.nereids.rules.rewrite.PushProjectThroughUnion; @@ -83,6 +85,7 @@ import org.apache.doris.nereids.rules.rewrite.PushdownLimit; import org.apache.doris.nereids.rules.rewrite.PushdownTopNThroughWindow; import org.apache.doris.nereids.rules.rewrite.ReorderJoin; +import org.apache.doris.nereids.rules.rewrite.RewriteCteChildren; import org.apache.doris.nereids.rules.rewrite.SemiJoinCommute; import org.apache.doris.nereids.rules.rewrite.SimplifyAggGroupBy; import org.apache.doris.nereids.rules.rewrite.SplitLimit; @@ -97,15 +100,14 @@ import org.apache.doris.nereids.rules.rewrite.mv.SelectMaterializedIndexWithoutAggregate; import java.util.List; +import java.util.stream.Collectors; /** - * Apply rules to optimize logical plan. + * Apply rules to rewrite logical plan. */ public class Rewriter extends AbstractBatchJobExecutor { - public static final List REWRITE_JOBS = jobs( - bottomUp(new InlineCTE()), - custom(RuleType.ADD_DEFAULT_LIMIT, AddDefaultLimit::new), + private static final List CTE_CHILDREN_REWRITE_JOBS = jobs( topic("Plan Normalization", topDown( new EliminateOrderByConstant(), @@ -129,6 +131,7 @@ public class Rewriter extends AbstractBatchJobExecutor { new ExtractSingleTableExpressionFromDisjunction() ) ), + // subquery unnesting relay on ExpressionNormalization to extract common factor expression topic("Subquery unnesting", costBased( custom(RuleType.AGG_SCALAR_SUBQUERY_TO_WINDOW_FUNCTION, @@ -273,11 +276,6 @@ public class Rewriter extends AbstractBatchJobExecutor { bottomUp(RuleSet.PUSH_DOWN_FILTERS), custom(RuleType.ELIMINATE_UNNECESSARY_PROJECT, EliminateUnnecessaryProject::new) ), - topic("Match expression check", - topDown( - new CheckMatchExpression() - ) - ), // this rule batch must keep at the end of rewrite to do some plan check topic("Final rewrite and check", custom(RuleType.ENSURE_PROJECT_ON_TOP_JOIN, EnsureProjectOnTopJoin::new), @@ -286,30 +284,73 @@ public class Rewriter extends AbstractBatchJobExecutor { new MergeProjects() ), custom(RuleType.ADJUST_CONJUNCTS_RETURN_TYPE, AdjustConjunctsReturnType::new), - custom(RuleType.ADJUST_NULLABLE, AdjustNullable::new), bottomUp( new ExpressionRewrite(CheckLegalityAfterRewrite.INSTANCE), new CheckMatchExpression(), new CheckMultiDistinct(), new CheckAfterRewrite() - )), - topic("MATERIALIZED CTE", topDown( + ) + ), + topic("Push project and filter on cte consumer to cte producer", + topDown( new CollectFilterAboveConsumer(), - new CollectProjectAboveConsumer(), - new BuildCTEAnchorAndCTEProducer()), - topDown(new CTEProducerRewrite())) + new CollectProjectAboveConsumer() + ) + ) ); + private static final List WHOLE_TREE_REWRITE_JOBS + = getWholeTreeRewriteJobs(true); + + private static final List WHOLE_TREE_REWRITE_JOBS_WITHOUT_COST_BASED + = getWholeTreeRewriteJobs(false); + private final List rewriteJobs; - public Rewriter(CascadesContext cascadesContext) { + private Rewriter(CascadesContext cascadesContext, List rewriteJobs) { super(cascadesContext); - this.rewriteJobs = REWRITE_JOBS; + this.rewriteJobs = rewriteJobs; } - public Rewriter(CascadesContext cascadesContext, List rewriteJobs) { - super(cascadesContext); - this.rewriteJobs = rewriteJobs; + public static Rewriter getWholeTreeRewriterWithoutCostBasedJobs(CascadesContext cascadesContext) { + return new Rewriter(cascadesContext, WHOLE_TREE_REWRITE_JOBS_WITHOUT_COST_BASED); + } + + public static Rewriter getWholeTreeRewriter(CascadesContext cascadesContext) { + return new Rewriter(cascadesContext, WHOLE_TREE_REWRITE_JOBS); + } + + public static Rewriter getCteChildrenRewriter(CascadesContext cascadesContext, List jobs) { + return new Rewriter(cascadesContext, jobs); + } + + public static Rewriter getWholeTreeRewriterWithCustomJobs(CascadesContext cascadesContext, List jobs) { + return new Rewriter(cascadesContext, getWholeTreeRewriteJobs(jobs)); + } + + private static List getWholeTreeRewriteJobs(boolean withCostBased) { + List withoutCostBased = Rewriter.CTE_CHILDREN_REWRITE_JOBS.stream() + .filter(j -> !(j instanceof CostBasedRewriteJob)) + .collect(Collectors.toList()); + return getWholeTreeRewriteJobs(withCostBased ? CTE_CHILDREN_REWRITE_JOBS : withoutCostBased); + } + + private static List getWholeTreeRewriteJobs(List jobs) { + return jobs( + topic("cte inline and pull up all cte anchor", + custom(RuleType.PULL_UP_CTE_ANCHOR, PullUpCteAnchor::new), + custom(RuleType.CTE_INLINE, CTEInline::new) + ), + topic("process limit session variables", + custom(RuleType.ADD_DEFAULT_LIMIT, AddDefaultLimit::new) + ), + topic("rewrite cte sub-tree", + custom(RuleType.REWRITE_CTE_CHILDREN, () -> new RewriteCteChildren(jobs)) + ), + topic("whole plan check", + custom(RuleType.ADJUST_NULLABLE, AdjustNullable::new) + ) + ); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CostBasedRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CostBasedRewriteJob.java index 72f5187cfd726c..2e5132f4ddd4ed 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CostBasedRewriteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/CostBasedRewriteJob.java @@ -24,6 +24,8 @@ import org.apache.doris.nereids.jobs.executor.Optimizer; import org.apache.doris.nereids.jobs.executor.Rewriter; import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.logging.log4j.LogManager; import org.apache.logging.log4j.Logger; @@ -48,37 +50,29 @@ public CostBasedRewriteJob(List rewriteJobs) { @Override public void execute(JobContext jobContext) { - CascadesContext cascadesContext = jobContext.getCascadesContext(); - CascadesContext skipCboRuleCtx = CascadesContext.newRewriteContext( - cascadesContext, cascadesContext.getRewritePlan(), - cascadesContext.getCurrentJobContext().getRequiredProperties()); - CascadesContext applyCboRuleCtx = CascadesContext.newRewriteContext( - cascadesContext, cascadesContext.getRewritePlan(), - cascadesContext.getCurrentJobContext().getRequiredProperties()); + CascadesContext currentCtx = jobContext.getCascadesContext(); + CascadesContext skipCboRuleCtx = CascadesContext.newCurrentTreeContext(currentCtx); + CascadesContext applyCboRuleCtx = CascadesContext.newCurrentTreeContext(currentCtx); // execute cbo rule on one candidate - new Rewriter(applyCboRuleCtx, rewriteJobs).execute(); + Rewriter.getCteChildrenRewriter(applyCboRuleCtx, rewriteJobs).execute(); if (skipCboRuleCtx.getRewritePlan().deepEquals(applyCboRuleCtx.getRewritePlan())) { // this means rewrite do not do anything return; } - // Do rewrite on 2 candidates - new Rewriter(skipCboRuleCtx, jobContext.getRemainJobs()).execute(); - new Rewriter(applyCboRuleCtx, jobContext.getRemainJobs()).execute(); - // Do optimize on 2 candidates - new Optimizer(skipCboRuleCtx).execute(); - new Optimizer(applyCboRuleCtx).execute(); - Optional> skipCboRuleCost = skipCboRuleCtx.getMemo().getRoot() - .getLowestCostPlan(skipCboRuleCtx.getCurrentJobContext().getRequiredProperties()); - Optional> appliedCboRuleCost = applyCboRuleCtx.getMemo().getRoot() - .getLowestCostPlan(applyCboRuleCtx.getCurrentJobContext().getRequiredProperties()); + + // compare two candidates + Optional> skipCboRuleCost = getCost(currentCtx, skipCboRuleCtx, jobContext); + Optional> appliedCboRuleCost = getCost(currentCtx, applyCboRuleCtx, jobContext); // If one of them optimize failed, just return if (!skipCboRuleCost.isPresent() || !appliedCboRuleCost.isPresent()) { - LOG.warn("Cbo rewrite execute failed"); + LOG.warn("Cbo rewrite execute failed on sql: {}, jobs are {}, plan is {}.", + currentCtx.getStatementContext().getOriginStatement().originStmt, + rewriteJobs, currentCtx.getRewritePlan()); return; } // If the candidate applied cbo rule is better, replace the original plan with it. if (appliedCboRuleCost.get().first.getValue() < skipCboRuleCost.get().first.getValue()) { - cascadesContext.setRewritePlan(applyCboRuleCtx.getRewritePlan()); + currentCtx.setRewritePlan(applyCboRuleCtx.getRewritePlan()); } } @@ -87,4 +81,27 @@ public boolean isOnce() { // TODO: currently, we do not support execute it more than once. return true; } + + private Optional> getCost(CascadesContext currentCtx, + CascadesContext cboCtx, JobContext jobContext) { + // Do subtree rewrite + Rewriter.getCteChildrenRewriter(cboCtx, jobContext.getRemainJobs()).execute(); + CascadesContext rootCtx = currentCtx.getRoot(); + if (rootCtx.getRewritePlan() instanceof LogicalCTEAnchor) { + // set subtree rewrite cache + currentCtx.getStatementContext().getRewrittenCtePlan() + .put(currentCtx.getCurrentTree().orElse(null), (LogicalPlan) cboCtx.getRewritePlan()); + // Do Whole tree rewrite + CascadesContext rootCtxCopy = CascadesContext.newCurrentTreeContext(rootCtx); + Rewriter.getWholeTreeRewriterWithoutCostBasedJobs(rootCtxCopy).execute(); + // Do optimize + new Optimizer(rootCtxCopy).execute(); + return rootCtxCopy.getMemo().getRoot().getLowestCostPlan( + rootCtxCopy.getCurrentJobContext().getRequiredProperties()); + } else { + new Optimizer(cboCtx).execute(); + return cboCtx.getMemo().getRoot().getLowestCostPlan( + cboCtx.getCurrentJobContext().getRequiredProperties()); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java index 36d168f5c17489..4f623e5450060f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteBottomUpJob.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.jobs.JobType; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; import java.util.List; import java.util.Objects; @@ -102,7 +103,10 @@ private void traverseClearState() { Plan child = children.get(i); RewriteJobContext childRewriteJobContext = new RewriteJobContext( child, clearedStateContext, i, false); - pushJob(new PlanTreeRewriteBottomUpJob(childRewriteJobContext, context, rules)); + // NOTICE: this relay on pull up cte anchor + if (!(rewriteJobContext.plan instanceof LogicalCTEAnchor)) { + pushJob(new PlanTreeRewriteBottomUpJob(childRewriteJobContext, context, rules)); + } } } @@ -142,7 +146,10 @@ private void ensureChildrenRewritten() { // we should transform this new plan nodes too. RewriteJobContext childRewriteJobContext = new RewriteJobContext( child, rewriteJobContext, i, false); - pushJob(new PlanTreeRewriteBottomUpJob(childRewriteJobContext, context, rules)); + // NOTICE: this relay on pull up cte anchor + if (!(rewriteJobContext.plan instanceof LogicalCTEAnchor)) { + pushJob(new PlanTreeRewriteBottomUpJob(childRewriteJobContext, context, rules)); + } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java index 3cb554d45a13a5..d8dba41b3788bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/PlanTreeRewriteTopDownJob.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.jobs.JobType; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; import java.util.List; import java.util.Objects; @@ -59,7 +60,10 @@ public void execute() { for (int i = children.size() - 1; i >= 0; i--) { RewriteJobContext childRewriteJobContext = new RewriteJobContext( children.get(i), newRewriteJobContext, i, false); - pushJob(new PlanTreeRewriteTopDownJob(childRewriteJobContext, context, rules)); + // NOTICE: this relay on pull up cte anchor + if (!(rewriteJobContext.plan instanceof LogicalCTEAnchor)) { + pushJob(new PlanTreeRewriteTopDownJob(childRewriteJobContext, context, rules)); + } } } else { // All the children part are already visited. Just link the children plan to the current node. diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java index 8547972cce7964..23db791e9bf3dd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/jobs/rewrite/RootPlanTreeRewriteJob.java @@ -50,8 +50,6 @@ public void execute(JobContext context) { context.getScheduleContext().pushJob(rewriteJob); cascadesContext.getJobScheduler().executeJobPool(cascadesContext); - - cascadesContext.setCurrentRootRewriteJobContext(null); } @Override @@ -72,7 +70,6 @@ public static class RootRewriteJobContext extends RewriteJobContext { RootRewriteJobContext(Plan plan, boolean childrenVisited, JobContext jobContext) { super(plan, null, -1, childrenVisited); this.jobContext = Objects.requireNonNull(jobContext, "jobContext cannot be null"); - jobContext.getCascadesContext().setCurrentRootRewriteJobContext(this); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java index 0c82914366483b..67534014ce6581 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java @@ -19,9 +19,6 @@ import org.apache.doris.common.IdGenerator; import org.apache.doris.common.Pair; -import org.apache.doris.nereids.CTEContext; -import org.apache.doris.nereids.CascadesContext; -import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.cost.Cost; import org.apache.doris.nereids.cost.CostCalculator; import org.apache.doris.nereids.metrics.EventChannel; @@ -304,17 +301,6 @@ public Plan copyOut(GroupExpression logicalExpression, boolean includeGroupExpre return planWithChildren.withGroupExpression(groupExpression); } - /** - * Utility function to create a new {@link CascadesContext} with this Memo. - */ - public CascadesContext newCascadesContext(StatementContext statementContext) { - return new CascadesContext(null, this, statementContext, PhysicalProperties.ANY); - } - - public CascadesContext newCascadesContext(StatementContext statementContext, CTEContext cteContext) { - return new CascadesContext(null, this, statementContext, cteContext, PhysicalProperties.ANY); - } - /** * init memo by a first plan. * @param plan first plan diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index bb1181e008975c..696ac16f6f36bc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -161,6 +161,7 @@ import org.apache.doris.nereids.trees.expressions.OrderExpression; import org.apache.doris.nereids.trees.expressions.Regexp; import org.apache.doris.nereids.trees.expressions.ScalarSubquery; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.Subtract; import org.apache.doris.nereids.trees.expressions.TVFProperties; import org.apache.doris.nereids.trees.expressions.TimestampArithmetic; @@ -244,7 +245,6 @@ import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.types.coercion.CharacterType; import org.apache.doris.nereids.util.ExpressionUtils; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.policy.FilterType; import org.apache.doris.policy.PolicyTypeEnum; import org.apache.doris.qe.ConnectContext; @@ -335,7 +335,7 @@ public LogicalPlan visitInsertIntoQuery(InsertIntoQueryContext ctx) { @Override public LogicalPlan visitUpdate(UpdateContext ctx) { LogicalPlan query = withCheckPolicy(new UnboundRelation( - RelationUtil.newRelationId(), visitMultipartIdentifier(ctx.tableName))); + StatementScopeIdGenerator.newRelationId(), visitMultipartIdentifier(ctx.tableName))); query = withTableAlias(query, ctx.tableAlias()); if (ctx.fromClause() != null) { query = withRelations(query, ctx.fromClause().relation()); @@ -354,7 +354,7 @@ public LogicalPlan visitDelete(DeleteContext ctx) { List tableName = visitMultipartIdentifier(ctx.tableName); List partitions = ctx.partition == null ? ImmutableList.of() : visitIdentifierList(ctx.partition); LogicalPlan query = withTableAlias(withCheckPolicy( - new UnboundRelation(RelationUtil.newRelationId(), tableName)), ctx.tableAlias()); + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), tableName)), ctx.tableAlias()); if (ctx.USING() != null) { query = withRelations(query, ctx.relation()); } @@ -582,7 +582,8 @@ public LogicalPlan visitTableName(TableNameContext ctx) { } LogicalPlan checkedRelation = withCheckPolicy( - new UnboundRelation(RelationUtil.newRelationId(), tableId, partitionNames, isTempPart, relationHints)); + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), + tableId, partitionNames, isTempPart, relationHints)); LogicalPlan plan = withTableAlias(checkedRelation, ctx.tableAlias()); for (LateralViewContext lateralViewContext : ctx.lateralView()) { plan = withGenerate(plan, lateralViewContext); @@ -613,7 +614,7 @@ public LogicalPlan visitTableValuedFunction(TableValuedFunctionContext ctx) { String value = parseTVFPropertyItem(argument.value); map.put(key, value); } - LogicalPlan relation = new UnboundTVFRelation(RelationUtil.newRelationId(), + LogicalPlan relation = new UnboundTVFRelation(StatementScopeIdGenerator.newRelationId(), functionName, new TVFProperties(map.build())); return withTableAlias(relation, ctx.tableAlias()); }); @@ -1488,7 +1489,7 @@ private UnboundOneRowRelation withOneRowRelation(SelectColumnClauseContext selec return ParserUtils.withOrigin(selectCtx, () -> { // fromClause does not exists. List projects = getNamedExpressions(selectCtx.namedExpressionSeq()); - return new UnboundOneRowRelation(RelationUtil.newRelationId(), projects); + return new UnboundOneRowRelation(StatementScopeIdGenerator.newRelationId(), projects); }); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterContext.java index 76fb3311fe5fa6..5aab5ab5dd1613 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterContext.java @@ -26,8 +26,8 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.physical.AbstractPhysicalJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEProducer; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashJoin; @@ -63,7 +63,7 @@ public class RuntimeFilterContext { private final Map> joinToTargetExprId = Maps.newHashMap(); // olap scan node that contains target of a runtime filter. - private final Map> targetOnOlapScanNodeMap = Maps.newHashMap(); + private final Map> targetOnOlapScanNodeMap = Maps.newHashMap(); private final List legacyFilters = Lists.newArrayList(); @@ -157,7 +157,7 @@ public void removeFilter(ExprId targetId, PhysicalHashJoin builderNode) { } } - public void setTargetsOnScanNode(ObjectId id, Slot slot) { + public void setTargetsOnScanNode(RelationId id, Slot slot) { this.targetOnOlapScanNodeMap.computeIfAbsent(id, k -> Lists.newArrayList()).add(slot); } @@ -186,7 +186,7 @@ public Map> getTargetExprIdToFilter() { return targetExprIdToFilter; } - public Map> getTargetOnOlapScanNodeMap() { + public Map> getTargetOnOlapScanNodeMap() { return targetOnOlapScanNodeMap; } 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 537b156b81e388..1e5575e4ac1109 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 @@ -203,7 +203,7 @@ public PhysicalPlan visitPhysicalNestedLoopJoin(PhysicalNestedLoopJoin filter, @Override public PhysicalRelation visitPhysicalScan(PhysicalRelation scan, CascadesContext context) { RuntimeFilterContext rfCtx = context.getRuntimeFilterContext(); - List slots = rfCtx.getTargetOnOlapScanNodeMap().get(scan.getId()); + List slots = rfCtx.getTargetOnOlapScanNodeMap().get(scan.getRelationId()); if (slots != null) { for (Slot slot : slots) { //if this scan node is the target of any effective RF, it is effective source diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index f033c6d465873a..acdf9f790a5103 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -42,8 +42,8 @@ import org.apache.doris.nereids.rules.implementation.AggregateStrategies; import org.apache.doris.nereids.rules.implementation.LogicalAssertNumRowsToPhysicalAssertNumRows; import org.apache.doris.nereids.rules.implementation.LogicalCTEAnchorToPhysicalCTEAnchor; -import org.apache.doris.nereids.rules.implementation.LogicalCTEConsumeToPhysicalCTEConsume; -import org.apache.doris.nereids.rules.implementation.LogicalCTEProduceToPhysicalCTEProduce; +import org.apache.doris.nereids.rules.implementation.LogicalCTEConsumerToPhysicalCTEConsumer; +import org.apache.doris.nereids.rules.implementation.LogicalCTEProducerToPhysicalCTEProducer; import org.apache.doris.nereids.rules.implementation.LogicalEmptyRelationToPhysicalEmptyRelation; import org.apache.doris.nereids.rules.implementation.LogicalEsScanToPhysicalEsScan; import org.apache.doris.nereids.rules.implementation.LogicalExceptToPhysicalExcept; @@ -76,8 +76,6 @@ import org.apache.doris.nereids.rules.rewrite.PushdownAliasThroughJoin; import org.apache.doris.nereids.rules.rewrite.PushdownExpressionsInHashCondition; import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughAggregation; -import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughCTE; -import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughCTEAnchor; import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughJoin; import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughProject; import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughRepeat; @@ -85,8 +83,6 @@ import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughSort; import org.apache.doris.nereids.rules.rewrite.PushdownFilterThroughWindow; import org.apache.doris.nereids.rules.rewrite.PushdownJoinOtherCondition; -import org.apache.doris.nereids.rules.rewrite.PushdownProjectThroughCTE; -import org.apache.doris.nereids.rules.rewrite.PushdownProjectThroughCTEAnchor; import org.apache.doris.nereids.rules.rewrite.PushdownProjectThroughLimit; import com.google.common.collect.ImmutableList; @@ -133,15 +129,11 @@ public class RuleSet { new MergeFilters(), new MergeGenerates(), new MergeLimits(), - new PushdownFilterThroughCTE(), - new PushdownProjectThroughCTE(), - new PushdownFilterThroughCTEAnchor(), - new PushdownProjectThroughCTEAnchor(), new PushdownAliasThroughJoin()); public static final List IMPLEMENTATION_RULES = planRuleFactories() - .add(new LogicalCTEProduceToPhysicalCTEProduce()) - .add(new LogicalCTEConsumeToPhysicalCTEConsume()) + .add(new LogicalCTEProducerToPhysicalCTEProducer()) + .add(new LogicalCTEConsumerToPhysicalCTEConsumer()) .add(new LogicalCTEAnchorToPhysicalCTEAnchor()) .add(new LogicalRepeatToPhysicalRepeat()) .add(new LogicalFilterToPhysicalFilter()) 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 c3a6012b02dfe2..8027ff7b0249db 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 @@ -70,7 +70,7 @@ public enum RuleType { PROJECT_TO_GLOBAL_AGGREGATE(RuleTypeClass.REWRITE), PROJECT_WITH_DISTINCT_TO_AGGREGATE(RuleTypeClass.REWRITE), AVG_DISTINCT_TO_SUM_DIV_COUNT(RuleTypeClass.REWRITE), - REGISTER_CTE(RuleTypeClass.REWRITE), + ANALYZE_CTE(RuleTypeClass.REWRITE), RELATION_AUTHENTICATION(RuleTypeClass.VALIDATION), ADJUST_NULLABLE_FOR_PROJECT_SLOT(RuleTypeClass.REWRITE), @@ -233,15 +233,12 @@ public enum RuleType { // ensure having project on the top join ENSURE_PROJECT_ON_TOP_JOIN(RuleTypeClass.REWRITE), - BUILD_CTE_ANCHOR_AND_CTE_PRODUCER(RuleTypeClass.REWRITE), + PULL_UP_CTE_ANCHOR(RuleTypeClass.REWRITE), + CTE_INLINE(RuleTypeClass.REWRITE), + REWRITE_CTE_CHILDREN(RuleTypeClass.REWRITE), COLLECT_FILTER_ON_CONSUMER(RuleTypeClass.REWRITE), COLLECT_PROJECT_ABOVE_CONSUMER(RuleTypeClass.REWRITE), - COLLECT_PROJECT_ABOVE_FILTER_CONSUMER(RuleTypeClass.REWRITE), - CTE_PRODUCER_REWRITE(RuleTypeClass.REWRITE), - PUSH_DOWN_PROJECT_THROUGH_CTE(RuleTypeClass.REWRITE), - PUSH_DOWN_PROJECT_THROUGH_CTE_ANCHOR(RuleTypeClass.REWRITE), - INLINE_CTE(RuleTypeClass.REWRITE), REWRITE_SENTINEL(RuleTypeClass.REWRITE), // exploration rules @@ -285,8 +282,8 @@ public enum RuleType { LOGICAL_JOIN_TO_NESTED_LOOP_JOIN_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_PROJECT_TO_PHYSICAL_PROJECT_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_FILTER_TO_PHYSICAL_FILTER_RULE(RuleTypeClass.IMPLEMENTATION), - LOGICAL_CTE_PRODUCE_TO_PHYSICAL_CTE_PRODUCER_RULE(RuleTypeClass.IMPLEMENTATION), - LOGICAL_CTE_CONSUME_TO_PHYSICAL_CTE_CONSUMER_RULE(RuleTypeClass.IMPLEMENTATION), + LOGICAL_CTE_PRODUCER_TO_PHYSICAL_CTE_PRODUCER_RULE(RuleTypeClass.IMPLEMENTATION), + LOGICAL_CTE_CONSUMER_TO_PHYSICAL_CTE_CONSUMER_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_CTE_ANCHOR_TO_PHYSICAL_CTE_ANCHOR_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_SORT_TO_PHYSICAL_QUICK_SORT_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_TOP_N_TO_PHYSICAL_TOP_N_RULE(RuleTypeClass.IMPLEMENTATION), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java new file mode 100644 index 00000000000000..65d9607a1ce2e2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTE.java @@ -0,0 +1,127 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.analysis; + +import org.apache.doris.common.Pair; +import org.apache.doris.nereids.CTEContext; +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; + +import com.google.common.collect.ImmutableList; + +import java.util.ArrayList; +import java.util.HashSet; +import java.util.List; +import java.util.Set; + +/** + * Register CTE, includes checking columnAliases, checking CTE name, analyzing each CTE and store the + * analyzed logicalPlan of CTE's query in CTEContext; + * A LogicalProject node will be added to the root of the initial logicalPlan if there exist columnAliases. + * Node LogicalCTE will be eliminated after registering. + */ +public class AnalyzeCTE extends OneAnalysisRuleFactory { + + @Override + public Rule build() { + return logicalCTE().thenApply(ctx -> { + LogicalCTE logicalCTE = ctx.root; + + // step 1. analyzed all cte plan + Pair>> result = analyzeCte(logicalCTE, ctx.cascadesContext); + CascadesContext outerCascadesCtx = CascadesContext.newContextWithCteContext( + ctx.cascadesContext, logicalCTE.child(), result.first); + outerCascadesCtx.newAnalyzer().analyze(); + Plan root = outerCascadesCtx.getRewritePlan(); + // should construct anchor from back to front, because the cte behind depends on the front + for (int i = result.second.size() - 1; i >= 0; i--) { + root = new LogicalCTEAnchor<>(result.second.get(i).getCteId(), result.second.get(i), root); + } + return root; + }).toRule(RuleType.ANALYZE_CTE); + } + + /** + * register and store CTEs in CTEContext + */ + private Pair>> analyzeCte( + LogicalCTE logicalCTE, CascadesContext cascadesContext) { + CTEContext outerCteCtx = cascadesContext.getCteContext(); + List> aliasQueries = logicalCTE.getAliasQueries(); + List> cteProducerPlans = new ArrayList<>(); + for (LogicalSubQueryAlias aliasQuery : aliasQueries) { + String cteName = aliasQuery.getAlias(); + if (outerCteCtx.containsCTE(cteName)) { + throw new AnalysisException("CTE name [" + cteName + "] cannot be used more than once."); + } + + // we should use a chain to ensure visible of cte + CTEContext innerCteCtx = outerCteCtx; + + LogicalPlan parsedCtePlan = (LogicalPlan) aliasQuery.child(); + CascadesContext innerCascadesCtx = CascadesContext.newContextWithCteContext( + cascadesContext, parsedCtePlan, innerCteCtx); + innerCascadesCtx.newAnalyzer().analyze(); + LogicalPlan analyzedCtePlan = (LogicalPlan) innerCascadesCtx.getRewritePlan(); + checkColumnAlias(aliasQuery, analyzedCtePlan.getOutput()); + CTEId cteId = cascadesContext.getStatementContext().getNextCTEId(); + LogicalSubQueryAlias logicalSubQueryAlias = + aliasQuery.withChildren(ImmutableList.of(analyzedCtePlan)); + outerCteCtx = new CTEContext(cteId, logicalSubQueryAlias, outerCteCtx); + outerCteCtx.setAnalyzedPlan(logicalSubQueryAlias); + cteProducerPlans.add(new LogicalCTEProducer<>(cteId, logicalSubQueryAlias)); + } + return Pair.of(outerCteCtx, cteProducerPlans); + } + + /** + * check columnAliases' size and name + */ + private void checkColumnAlias(LogicalSubQueryAlias aliasQuery, List outputSlots) { + if (aliasQuery.getColumnAliases().isPresent()) { + List columnAlias = aliasQuery.getColumnAliases().get(); + // if the size of columnAlias is smaller than outputSlots' size, we will replace the corresponding number + // of front slots with columnAlias. + if (columnAlias.size() > outputSlots.size()) { + throw new AnalysisException("CTE [" + aliasQuery.getAlias() + "] returns " + + columnAlias.size() + " columns, but " + outputSlots.size() + " labels were specified." + + " The number of column labels must be smaller or equal to the number of returned columns."); + } + + Set names = new HashSet<>(); + // column alias cannot be used more than once + columnAlias.forEach(alias -> { + if (names.contains(alias.toLowerCase())) { + throw new AnalysisException("Duplicated CTE column alias:" + + " [" + alias.toLowerCase() + "] in CTE [" + aliasQuery.getAlias() + "]"); + } + names.add(alias); + }); + } + } +} 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 bde3f2c8efaec1..9cb49f1c763040 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 @@ -484,7 +484,7 @@ protected boolean condition(Rule rule, Plan plan) { .map(project -> bindSlot(project, ImmutableList.of(), ctx.cascadesContext)) .map(project -> bindFunction(project, ctx.cascadesContext)) .collect(Collectors.toList()); - return new LogicalOneRowRelation(projects); + return new LogicalOneRowRelation(oneRowRelation.getRelationId(), projects); }) ), RuleType.BINDING_SET_OPERATION_SLOT.build( @@ -508,27 +508,18 @@ protected boolean condition(Rule rule, Plan plan) { } // 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(); + List> childrenProjections = setOperation.collectChildrenProjections(); ImmutableList.Builder newChildren = ImmutableList.builder(); - for (int i = 0; i < castExpressions.size(); i++) { - if (castExpressions.stream().allMatch(SlotReference.class::isInstance)) { + for (int i = 0; i < childrenProjections.size(); i++) { + if (childrenProjections.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); + newChildren.add(new LogicalProject<>(childrenProjections.get(i), setOperation.child(i))); } } - List newOutputs = setOperation.buildNewOutputs(castExpressions.get(0)); - return setOperation.withNewOutputs(newOutputs).withChildren(newChildren.build()); + setOperation = (LogicalSetOperation) setOperation.withChildren(newChildren.build()); + List newOutputs = setOperation.buildNewOutputs(); + return setOperation.withNewOutputs(newOutputs); }) ), RuleType.BINDING_GENERATE_SLOT.build( @@ -618,7 +609,6 @@ private List bindSlot( .collect(Collectors.toList()); } - @SuppressWarnings("unchecked") private E bindSlot(E expr, Plan input, CascadesContext cascadesContext) { return bindSlot(expr, input, cascadesContext, true, true); } @@ -700,7 +690,7 @@ private LogicalTVFRelation bindTableValuedFunction(UnboundTVFRelation unboundTVF if (!(function instanceof TableValuedFunction)) { throw new AnalysisException(function.toSql() + " is not a TableValuedFunction"); } - return new LogicalTVFRelation(unboundTVFRelation.getId(), (TableValuedFunction) function); + return new LogicalTVFRelation(unboundTVFRelation.getRelationId(), (TableValuedFunction) function); } private void checkSameNameSlot(List childOutputs, String subQueryAlias) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 0775fc7acbbf7f..66e8a7879e12a3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -65,20 +65,19 @@ import java.util.List; import java.util.Optional; import java.util.function.Function; -import javax.annotation.Nullable; /** * Rule to bind relations in query plan. */ public class BindRelation extends OneAnalysisRuleFactory { - private CustomTableResolver customTableResolver; + private final Optional customTableResolver; public BindRelation() { - this(null); + this(Optional.empty()); } - public BindRelation(@Nullable CustomTableResolver customTableResolver) { + public BindRelation(Optional customTableResolver) { this.customTableResolver = customTableResolver; } @@ -123,13 +122,10 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe // check if it is a CTE's name CTEContext cteContext = cascadesContext.getCteContext().findCTEContext(tableName).orElse(null); if (cteContext != null) { - Optional analyzedCte = cteContext.getReuse(tableName); + Optional analyzedCte = cteContext.getAnalyzedCTEPlan(tableName); if (analyzedCte.isPresent()) { - LogicalCTEConsumer logicalCTEConsumer = - new LogicalCTEConsumer(Optional.empty(), Optional.empty(), - analyzedCte.get(), cteContext.getCteId(), tableName); - cascadesContext.putCTEIdToConsumer(logicalCTEConsumer); - return logicalCTEConsumer; + return new LogicalCTEConsumer(unboundRelation.getRelationId(), + cteContext.getCteId(), tableName, analyzedCte.get()); } } List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), @@ -138,12 +134,14 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe if (cascadesContext.getTables() != null) { table = cascadesContext.getTableByName(tableName); } - if (customTableResolver != null) { - table = customTableResolver.apply(tableQualifier); + if (table == null) { + if (customTableResolver.isPresent()) { + table = customTableResolver.get().apply(tableQualifier); + } } + // In some cases even if we have already called the "cascadesContext.getTableByName", + // it also gets the null. So, we just check it in the catalog again for safety. if (table == null) { - // In some cases even if we have already called the "cascadesContext.getTableByName", - // it also gets the null. So, we just check it in the catalog again for safety. table = RelationUtil.getTable(tableQualifier, cascadesContext.getConnectContext().getEnv()); } @@ -155,9 +153,11 @@ private LogicalPlan bind(CascadesContext cascadesContext, UnboundRelation unboun List tableQualifier = RelationUtil.getQualifierName(cascadesContext.getConnectContext(), unboundRelation.getNameParts()); TableIf table = null; - if (customTableResolver != null) { - table = customTableResolver.apply(tableQualifier); + if (customTableResolver.isPresent()) { + table = customTableResolver.get().apply(tableQualifier); } + // In some cases even if we have already called the "cascadesContext.getTableByName", + // it also gets the null. So, we just check it in the catalog again for safety. if (table == null) { table = RelationUtil.getTable(tableQualifier, cascadesContext.getConnectContext().getEnv()); } @@ -168,10 +168,10 @@ private LogicalPlan makeOlapScan(TableIf table, UnboundRelation unboundRelation, LogicalOlapScan scan; List partIds = getPartitionIds(table, unboundRelation); if (!CollectionUtils.isEmpty(partIds)) { - scan = new LogicalOlapScan(RelationUtil.newRelationId(), + scan = new LogicalOlapScan(unboundRelation.getRelationId(), (OlapTable) table, ImmutableList.of(tableQualifier.get(1)), partIds, unboundRelation.getHints()); } else { - scan = new LogicalOlapScan(RelationUtil.newRelationId(), + scan = new LogicalOlapScan(unboundRelation.getRelationId(), (OlapTable) table, ImmutableList.of(tableQualifier.get(1)), unboundRelation.getHints()); } if (!Util.showHiddenColumns() && scan.getTable().hasDeleteSign() @@ -219,12 +219,13 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio case MAX_COMPUTE_EXTERNAL_TABLE: return new LogicalFileScan(RelationUtil.newRelationId(), (ExternalTable) table, tableQualifier); case SCHEMA: - return new LogicalSchemaScan(RelationUtil.newRelationId(), table, ImmutableList.of(dbName)); + return new LogicalSchemaScan(unboundRelation.getRelationId(), + table, ImmutableList.of(dbName)); case JDBC_EXTERNAL_TABLE: case JDBC: - return new LogicalJdbcScan(RelationUtil.newRelationId(), table, ImmutableList.of(dbName)); + return new LogicalJdbcScan(unboundRelation.getRelationId(), table, ImmutableList.of(dbName)); case ES_EXTERNAL_TABLE: - return new LogicalEsScan(RelationUtil.newRelationId(), + return new LogicalEsScan(unboundRelation.getRelationId(), (EsExternalTable) table, ImmutableList.of(dbName)); default: throw new AnalysisException("Unsupported tableType:" + table.getType()); @@ -245,7 +246,7 @@ private Plan parseAndAnalyzeHiveView(TableIf table, CascadesContext cascadesCont private Plan parseAndAnalyzeView(String viewSql, CascadesContext parentContext) { LogicalPlan parsedViewPlan = new NereidsParser().parseSingle(viewSql); - CascadesContext viewContext = CascadesContext.newRewriteContext( + CascadesContext viewContext = CascadesContext.initContext( parentContext.getStatementContext(), parsedViewPlan, PhysicalProperties.ANY); viewContext.newAnalyzer().analyze(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java index 7e7c1e55dd7955..a30cdfcbdd8502 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CheckAfterRewrite.java @@ -105,10 +105,14 @@ private void checkAllSlotReferenceFromChildren(Plan plan) { .collect(Collectors.toSet()); notFromChildren = removeValidSlotsNotFromChildren(notFromChildren, childrenOutput); if (!notFromChildren.isEmpty()) { - throw new AnalysisException(String.format("Input slot(s) not in child's output: %s in plan: %s", + throw new AnalysisException(String.format("Input slot(s) not in child's output: %s in plan: %s," + + " child output is: %s", StringUtils.join(notFromChildren.stream() - .map(ExpressionTrait::toSql) - .collect(Collectors.toSet()), ", "), plan)); + .map(ExpressionTrait::toString) + .collect(Collectors.toSet()), ", "), plan, + plan.children().stream() + .flatMap(child -> child.getOutput().stream()) + .collect(Collectors.toSet()))); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ColumnAliasGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ColumnAliasGenerator.java index 75e5e0cc5e15ab..2c3fa9cf25e023 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ColumnAliasGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ColumnAliasGenerator.java @@ -21,9 +21,6 @@ package org.apache.doris.nereids.rules.analysis; import org.apache.doris.common.AliasGenerator; -import org.apache.doris.nereids.StatementContext; - -import com.google.common.base.Preconditions; /** * Generate the table name required in the rewrite process. @@ -31,9 +28,7 @@ public class ColumnAliasGenerator extends AliasGenerator { private static final String DEFAULT_COL_ALIAS_PREFIX = "$c$"; - public ColumnAliasGenerator(StatementContext statementContext) { - Preconditions.checkNotNull(statementContext); + public ColumnAliasGenerator() { aliasPrefix = DEFAULT_COL_ALIAS_PREFIX; - usedAliases.addAll(statementContext.getColumnNames()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LogicalSubQueryAliasToLogicalProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LogicalSubQueryAliasToLogicalProject.java index b8a0566efb9c80..d37e57a02b356b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LogicalSubQueryAliasToLogicalProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/LogicalSubQueryAliasToLogicalProject.java @@ -19,6 +19,7 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.rules.rewrite.OneRewriteRuleFactory; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import com.google.common.collect.ImmutableList; @@ -29,7 +30,7 @@ *

* TODO: refactor group merge strategy to support the feature above */ -public class LogicalSubQueryAliasToLogicalProject extends OneAnalysisRuleFactory { +public class LogicalSubQueryAliasToLogicalProject extends OneRewriteRuleFactory { @Override public Rule build() { return RuleType.LOGICAL_SUB_QUERY_ALIAS_TO_LOGICAL_PROJECT.build( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/RegisterCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/RegisterCTE.java deleted file mode 100644 index 6f90d0544a5430..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/RegisterCTE.java +++ /dev/null @@ -1,127 +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.analysis; - -import org.apache.doris.nereids.CTEContext; -import org.apache.doris.nereids.CascadesContext; -import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.rules.Rule; -import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.expressions.CTEId; -import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; - -import com.google.common.collect.ImmutableList; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.List; -import java.util.Set; -import java.util.concurrent.Callable; - -/** - * Register CTE, includes checking columnAliases, checking CTE name, analyzing each CTE and store the - * analyzed logicalPlan of CTE's query in CTEContext; - * A LogicalProject node will be added to the root of the initial logicalPlan if there exist columnAliases. - * Node LogicalCTE will be eliminated after registering. - */ -public class RegisterCTE extends OneAnalysisRuleFactory { - - @Override - public Rule build() { - return logicalCTE().whenNot(LogicalCTE::isRegistered).thenApply(ctx -> { - LogicalCTE logicalCTE = ctx.root; - List> analyzedCTE = register(logicalCTE, ctx.cascadesContext); - return new LogicalCTE<>(analyzedCTE, logicalCTE.child(), true, - logicalCTE.getCteNameToId()); - }).toRule(RuleType.REGISTER_CTE); - } - - /** - * register and store CTEs in CTEContext - */ - private List> register(LogicalCTE logicalCTE, - CascadesContext cascadesContext) { - CTEContext cteCtx = cascadesContext.getCteContext(); - List> aliasQueries = logicalCTE.getAliasQueries(); - List> analyzedCTE = new ArrayList<>(); - for (LogicalSubQueryAlias aliasQuery : aliasQueries) { - String cteName = aliasQuery.getAlias(); - if (cteCtx.containsCTE(cteName)) { - throw new AnalysisException("CTE name [" + cteName + "] cannot be used more than once."); - } - - // we should use a chain to ensure visible of cte - CTEContext localCteContext = cteCtx; - - LogicalPlan parsedPlan = (LogicalPlan) aliasQuery.child(); - CascadesContext localCascadesContext = CascadesContext.newRewriteContext( - cascadesContext.getStatementContext(), parsedPlan, localCteContext); - localCascadesContext.newAnalyzer().analyze(); - LogicalPlan analyzedCteBody = (LogicalPlan) localCascadesContext.getRewritePlan(); - cascadesContext.putAllCTEIdToConsumer(localCascadesContext.getCteIdToConsumers()); - cascadesContext.putAllCTEIdToCTEClosure(localCascadesContext.getCteIdToCTEClosure()); - if (aliasQuery.getColumnAliases().isPresent()) { - checkColumnAlias(aliasQuery, analyzedCteBody.getOutput()); - } - CTEId cteId = logicalCTE.findCTEId(aliasQuery.getAlias()); - cteCtx = new CTEContext(aliasQuery, localCteContext, cteId); - - LogicalSubQueryAlias logicalSubQueryAlias = - aliasQuery.withChildren(ImmutableList.of(analyzedCteBody)); - cteCtx.setAnalyzedPlan(logicalSubQueryAlias); - Callable cteClosure = () -> { - CascadesContext localCascadesContextInClosure = CascadesContext.newRewriteContext( - cascadesContext.getStatementContext(), aliasQuery, localCteContext); - localCascadesContextInClosure.newAnalyzer().analyze(); - return (LogicalPlan) localCascadesContextInClosure.getRewritePlan(); - }; - cascadesContext.putCTEIdToCTEClosure(cteId, cteClosure); - analyzedCTE.add(logicalSubQueryAlias); - } - cascadesContext.setCteContext(cteCtx); - return analyzedCTE; - } - - /** - * check columnAliases' size and name - */ - private void checkColumnAlias(LogicalSubQueryAlias aliasQuery, List outputSlots) { - List columnAlias = aliasQuery.getColumnAliases().get(); - // if the size of columnAlias is smaller than outputSlots' size, we will replace the corresponding number - // of front slots with columnAlias. - if (columnAlias.size() > outputSlots.size()) { - throw new AnalysisException("CTE [" + aliasQuery.getAlias() + "] returns " + columnAlias.size() - + " columns, but " + outputSlots.size() + " labels were specified. The number of column labels must " - + "be smaller or equal to the number of returned columns."); - } - - Set names = new HashSet<>(); - // column alias cannot be used more than once - columnAlias.forEach(alias -> { - if (names.contains(alias.toLowerCase())) { - throw new AnalysisException("Duplicated CTE column alias: [" + alias.toLowerCase() - + "] in CTE [" + aliasQuery.getAlias() + "]"); - } - names.add(alias); - }); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SlotBinder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SlotBinder.java index f64ee4a6b70225..e5f13007bd762f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SlotBinder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SlotBinder.java @@ -30,7 +30,6 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import com.google.common.base.Preconditions; import org.apache.commons.lang3.StringUtils; import java.util.List; @@ -70,15 +69,10 @@ public Expression bind(Expression expression) { public Expression visitUnboundAlias(UnboundAlias unboundAlias, CascadesContext context) { Expression child = unboundAlias.child().accept(this, context); if (unboundAlias.getAlias().isPresent()) { - collectColumnNames(unboundAlias.getAlias().get()); return new Alias(child, unboundAlias.getAlias().get()); - } - if (child instanceof NamedExpression) { - collectColumnNames(((NamedExpression) child).getName()); + } else if (child instanceof NamedExpression) { return new Alias(child, ((NamedExpression) child).getName()); } else { - // TODO: resolve aliases - collectColumnNames(child.toSql()); return new Alias(child, child.toSql()); } } @@ -223,11 +217,4 @@ private List bindSlot(UnboundSlot unboundSlot, List boundSlots) { + StringUtils.join(nameParts, ".")); }).collect(Collectors.toList()); } - - private void collectColumnNames(String columnName) { - Preconditions.checkNotNull(getCascadesContext()); - if (!getCascadesContext().getStatementContext().getColumnNames().add(columnName)) { - throw new AnalysisException("Collect column name failed, columnName : " + columnName); - } - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubExprAnalyzer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubExprAnalyzer.java index 20e695109db06a..9b323b5ba43465 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubExprAnalyzer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/SubExprAnalyzer.java @@ -168,11 +168,11 @@ private void checkRootIsLimit(AnalyzedResult analyzedResult) { } private AnalyzedResult analyzeSubquery(SubqueryExpr expr) { - CascadesContext subqueryContext = CascadesContext.newRewriteContext(cascadesContext, expr.getQueryPlan()); + CascadesContext subqueryContext = CascadesContext.newContextWithCteContext( + cascadesContext, expr.getQueryPlan(), cascadesContext.getCteContext()); Scope subqueryScope = genScopeWithSubquery(expr); subqueryContext.setOuterScope(subqueryScope); subqueryContext.newAnalyzer().analyze(); - cascadesContext.putAllCTEIdToConsumer(subqueryContext.getCteIdToConsumers()); return new AnalyzedResult((LogicalPlan) subqueryContext.getRewritePlan(), subqueryScope.getCorrelatedSlots()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/UserAuthentication.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/UserAuthentication.java index 9ad3edc6834d04..17e32fa95aed89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/UserAuthentication.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/UserAuthentication.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.qe.ConnectContext; /** @@ -37,14 +38,15 @@ public class UserAuthentication extends OneAnalysisRuleFactory { @Override public Rule build() { return logicalRelation() - .thenApply(ctx -> checkPermission(ctx.root, ctx.connectContext)) + .when(CatalogRelation.class::isInstance) + .thenApply(ctx -> checkPermission((CatalogRelation) ctx.root, ctx.connectContext)) .toRule(RuleType.RELATION_AUTHENTICATION); } - private Plan checkPermission(LogicalRelation relation, ConnectContext connectContext) { + private Plan checkPermission(CatalogRelation relation, ConnectContext connectContext) { // do not check priv when replaying dump file if (connectContext.getSessionVariable().isPlayNereidsDump()) { - return relation; + return null; } TableIf table = relation.getTable(); if (table == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java index 83aaa362b24944..676c079e037579 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/expression/ExpressionRewrite.java @@ -111,7 +111,7 @@ public Rule build() { if (projects.equals(newProjects)) { return oneRowRelation; } - return new LogicalOneRowRelation(newProjects); + return new LogicalOneRowRelation(oneRowRelation.getRelationId(), newProjects); }).toRule(RuleType.REWRITE_ONE_ROW_RELATION_EXPRESSION); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEConsumeToPhysicalCTEConsume.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEConsumerToPhysicalCTEConsumer.java similarity index 73% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEConsumeToPhysicalCTEConsume.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEConsumerToPhysicalCTEConsumer.java index 041ec32f154e7b..a17aa0e8328ced 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEConsumeToPhysicalCTEConsume.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEConsumerToPhysicalCTEConsumer.java @@ -24,15 +24,16 @@ /** * Implementation rule that convert logical CTE consumer to physical CTE consumer. */ -public class LogicalCTEConsumeToPhysicalCTEConsume extends OneImplementationRuleFactory { +public class LogicalCTEConsumerToPhysicalCTEConsumer extends OneImplementationRuleFactory { @Override public Rule build() { return logicalCTEConsumer().then(cte -> new PhysicalCTEConsumer( - cte.getCteId(), - cte.getConsumerToProducerOutputMap(), - cte.getProducerToConsumerOutputMap(), - cte.getLogicalProperties() - ) - ).toRule(RuleType.LOGICAL_CTE_CONSUME_TO_PHYSICAL_CTE_CONSUMER_RULE); + cte.getRelationId(), + cte.getCteId(), + cte.getConsumerToProducerOutputMap(), + cte.getProducerToConsumerOutputMap(), + cte.getLogicalProperties() + ) + ).toRule(RuleType.LOGICAL_CTE_CONSUMER_TO_PHYSICAL_CTE_CONSUMER_RULE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEProduceToPhysicalCTEProduce.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEProducerToPhysicalCTEProducer.java similarity index 86% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEProduceToPhysicalCTEProduce.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEProducerToPhysicalCTEProducer.java index beeab98ed46b7a..bcc48f8d8b3f22 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEProduceToPhysicalCTEProduce.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalCTEProducerToPhysicalCTEProducer.java @@ -24,14 +24,13 @@ /** * Implementation rule that convert logical CTE producer to physical CTE producer. */ -public class LogicalCTEProduceToPhysicalCTEProduce extends OneImplementationRuleFactory { +public class LogicalCTEProducerToPhysicalCTEProducer extends OneImplementationRuleFactory { @Override public Rule build() { - return logicalCTEProducer().then(cte -> new PhysicalCTEProducer( + return logicalCTEProducer().then(cte -> new PhysicalCTEProducer<>( cte.getCteId(), - cte.getProjects(), cte.getLogicalProperties(), cte.child()) - ).toRule(RuleType.LOGICAL_CTE_PRODUCE_TO_PHYSICAL_CTE_PRODUCER_RULE); + ).toRule(RuleType.LOGICAL_CTE_PRODUCER_TO_PHYSICAL_CTE_PRODUCER_RULE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalEmptyRelationToPhysicalEmptyRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalEmptyRelationToPhysicalEmptyRelation.java index b48d07a4a73745..c3513ed5ba9ea1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalEmptyRelationToPhysicalEmptyRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalEmptyRelationToPhysicalEmptyRelation.java @@ -28,7 +28,8 @@ public class LogicalEmptyRelationToPhysicalEmptyRelation extends OneImplementati @Override public Rule build() { return logicalEmptyRelation() - .then(relation -> new PhysicalEmptyRelation(relation.getProjects(), relation.getLogicalProperties())) + .then(relation -> new PhysicalEmptyRelation(relation.getRelationId(), + relation.getProjects(), relation.getLogicalProperties())) .toRule(RuleType.LOGICAL_EMPTY_RELATION_TO_PHYSICAL_EMPTY_RELATION_RULE); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalEsScanToPhysicalEsScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalEsScanToPhysicalEsScan.java index 74ad0cc35656ea..1e4db2eba11810 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalEsScanToPhysicalEsScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalEsScanToPhysicalEsScan.java @@ -32,7 +32,7 @@ public class LogicalEsScanToPhysicalEsScan extends OneImplementationRuleFactory public Rule build() { return logicalEsScan().then(esScan -> new PhysicalEsScan( - esScan.getId(), + esScan.getRelationId(), esScan.getTable(), esScan.getQualifier(), DistributionSpecAny.INSTANCE, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java index 08e9e9e5038c31..c78d8174ac53ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalFileScanToPhysicalFileScan.java @@ -32,7 +32,7 @@ public class LogicalFileScanToPhysicalFileScan extends OneImplementationRuleFact public Rule build() { return logicalFileScan().then(fileScan -> new PhysicalFileScan( - fileScan.getId(), + fileScan.getRelationId(), fileScan.getTable(), fileScan.getQualifier(), DistributionSpecAny.INSTANCE, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalJdbcScanToPhysicalJdbcScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalJdbcScanToPhysicalJdbcScan.java index 6172229525079f..9c2c6518034ee9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalJdbcScanToPhysicalJdbcScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalJdbcScanToPhysicalJdbcScan.java @@ -17,7 +17,6 @@ package org.apache.doris.nereids.rules.implementation; -import org.apache.doris.nereids.properties.DistributionSpecAny; import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.physical.PhysicalJdbcScan; @@ -32,10 +31,9 @@ public class LogicalJdbcScanToPhysicalJdbcScan extends OneImplementationRuleFact public Rule build() { return logicalJdbcScan().then(jdbcScan -> new PhysicalJdbcScan( - jdbcScan.getId(), + jdbcScan.getRelationId(), jdbcScan.getTable(), jdbcScan.getQualifier(), - DistributionSpecAny.INSTANCE, Optional.empty(), jdbcScan.getLogicalProperties(), jdbcScan.getConjuncts()) 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 12399722af2e80..b578247d06c01d 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 @@ -50,7 +50,7 @@ public class LogicalOlapScanToPhysicalOlapScan extends OneImplementationRuleFact public Rule build() { return logicalOlapScan().then(olapScan -> new PhysicalOlapScan( - olapScan.getId(), + olapScan.getRelationId(), olapScan.getTable(), olapScan.getQualifier(), olapScan.getSelectedIndexId(), 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 0d007d69840f43..62cd2b705bb1e3 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 @@ -28,7 +28,7 @@ public class LogicalOneRowRelationToPhysicalOneRowRelation extends OneImplementa @Override public Rule build() { return logicalOneRowRelation() - .then(relation -> new PhysicalOneRowRelation( + .then(relation -> new PhysicalOneRowRelation(relation.getRelationId(), 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/LogicalSchemaScanToPhysicalSchemaScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalSchemaScanToPhysicalSchemaScan.java index cb4832067adce7..df459fcd45c0d9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalSchemaScanToPhysicalSchemaScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalSchemaScanToPhysicalSchemaScan.java @@ -30,7 +30,7 @@ public class LogicalSchemaScanToPhysicalSchemaScan extends OneImplementationRule @Override public Rule build() { return logicalSchemaScan().then(scan -> - new PhysicalSchemaScan(scan.getId(), + new PhysicalSchemaScan(scan.getRelationId(), scan.getTable(), scan.getQualifier(), Optional.empty(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalTVFRelationToPhysicalTVFRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalTVFRelationToPhysicalTVFRelation.java index 6b3099f83c9baf..4828f77d5eee9b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalTVFRelationToPhysicalTVFRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalTVFRelationToPhysicalTVFRelation.java @@ -28,7 +28,7 @@ public class LogicalTVFRelationToPhysicalTVFRelation extends OneImplementationRu @Override public Rule build() { return logicalTVFRelation() - .then(relation -> new PhysicalTVFRelation(relation.getId(), + .then(relation -> new PhysicalTVFRelation(relation.getRelationId(), relation.getFunction(), relation.getLogicalProperties())) .toRule(RuleType.LOGICAL_TVF_RELATION_TO_PHYSICAL_TVF_RELATION); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AddDefaultLimit.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java similarity index 72% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AddDefaultLimit.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java index 470b40be2840a0..1b5fd7314b9d07 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/AddDefaultLimit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java @@ -15,16 +15,16 @@ // specific language governing permissions and limitations // under the License. -package org.apache.doris.nereids.rules.analysis; +package org.apache.doris.nereids.rules.rewrite; import org.apache.doris.nereids.StatementContext; -import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; import org.apache.doris.nereids.jobs.JobContext; import org.apache.doris.nereids.trees.plans.LimitPhase; import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; @@ -53,27 +53,35 @@ public Plan visit(Plan plan, StatementContext context) { return plan; } + // should add limit under anchor to keep optimize opportunity @Override - public LogicalPlan visitLogicalLimit(LogicalLimit limit, StatementContext context) { - return limit; + public Plan visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, + StatementContext context) { + return cteAnchor.withChildren(cteAnchor.child(0), cteAnchor.child(1)); } + // we should keep that sink node is the top node of the plan tree. + // currently, it's one of the olap table sink and file sink. @Override - public LogicalPlan visitLogicalCTE(LogicalCTE cte, StatementContext context) { - Plan child = cte.child().accept(this, context); - return ((LogicalPlan) cte.withChildren(child)); + public Plan visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, + StatementContext context) { + Plan child = olapTableSink.child().accept(this, context); + return olapTableSink.withChildren(child); } - // we should keep that sink node is the top node of the plan tree. - // currently, it's one of the olap table sink and file sink. @Override - public LogicalPlan visitUnboundOlapTableSink(UnboundOlapTableSink sink, StatementContext context) { - Plan child = sink.child().accept(this, context); - return ((LogicalPlan) sink.withChildren(child)); + public Plan visitLogicalFileSink(LogicalFileSink fileSink, StatementContext context) { + Plan child = fileSink.child().accept(this, context); + return fileSink.withChildren(child); + } + + @Override + public Plan visitLogicalLimit(LogicalLimit limit, StatementContext context) { + return limit; } @Override - public LogicalPlan visitLogicalSort(LogicalSort sort, StatementContext context) { + public Plan visitLogicalSort(LogicalSort sort, StatementContext context) { ConnectContext ctx = context.getConnectContext(); if (ctx != null) { long defaultLimit = ctx.getSessionVariable().getDefaultOrderByLimit(); 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 d708d6226c7f60..eb907dcafedfa0 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 @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.functions.ExpressionTrait; @@ -30,15 +31,18 @@ import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalGenerate; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; 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.LogicalRepeat; import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; +import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; @@ -47,7 +51,9 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import com.google.common.collect.Lists; +import com.google.common.collect.Maps; +import java.util.LinkedHashMap; import java.util.List; import java.util.Map; import java.util.Set; @@ -57,70 +63,68 @@ * because some rule could change output's nullable. * So, we need add a rule to adjust all expression's nullable attribute after rewrite. */ -public class AdjustNullable extends DefaultPlanRewriter implements CustomRewriter { +public class AdjustNullable extends DefaultPlanRewriter> implements CustomRewriter { @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { - return plan.accept(this, null); + return plan.accept(this, Maps.newHashMap()); } @Override - public Plan visit(Plan plan, Void context) { - LogicalPlan logicalPlan = (LogicalPlan) super.visit(plan, context); - return logicalPlan.recomputeLogicalProperties(); + public Plan visit(Plan plan, Map replaceMap) { + LogicalPlan logicalPlan = (LogicalPlan) super.visit(plan, replaceMap); + logicalPlan = logicalPlan.recomputeLogicalProperties(); + logicalPlan.getOutputSet().forEach(s -> replaceMap.put(s.getExprId(), s)); + return logicalPlan; } @Override - public Plan visitLogicalAggregate(LogicalAggregate aggregate, Void context) { - aggregate = (LogicalAggregate) super.visit(aggregate, context); - Map exprIdSlotMap = collectChildrenOutputMap(aggregate); + public Plan visitLogicalAggregate(LogicalAggregate aggregate, Map replaceMap) { + aggregate = (LogicalAggregate) super.visit(aggregate, replaceMap); List newOutputs - = updateExpressions(aggregate.getOutputExpressions(), exprIdSlotMap); + = updateExpressions(aggregate.getOutputExpressions(), replaceMap); List newGroupExpressions - = updateExpressions(aggregate.getGroupByExpressions(), exprIdSlotMap); + = updateExpressions(aggregate.getGroupByExpressions(), replaceMap); + newOutputs.forEach(o -> replaceMap.put(o.getExprId(), o.toSlot())); return aggregate.withGroupByAndOutput(newGroupExpressions, newOutputs); } @Override - public Plan visitLogicalFilter(LogicalFilter filter, Void context) { - filter = (LogicalFilter) super.visit(filter, context); - Map exprIdSlotMap = collectChildrenOutputMap(filter); - Set conjuncts = updateExpressions(filter.getConjuncts(), exprIdSlotMap); + public Plan visitLogicalFilter(LogicalFilter filter, Map replaceMap) { + filter = (LogicalFilter) super.visit(filter, replaceMap); + Set conjuncts = updateExpressions(filter.getConjuncts(), replaceMap); return filter.withConjuncts(conjuncts).recomputeLogicalProperties(); } @Override - public Plan visitLogicalGenerate(LogicalGenerate generate, Void context) { - generate = (LogicalGenerate) super.visit(generate, context); - Map exprIdSlotMap = collectChildrenOutputMap(generate); - List newGenerators = updateExpressions(generate.getGenerators(), exprIdSlotMap); - return generate.withGenerators(newGenerators).recomputeLogicalProperties(); + public Plan visitLogicalGenerate(LogicalGenerate generate, Map replaceMap) { + generate = (LogicalGenerate) super.visit(generate, replaceMap); + List newGenerators = updateExpressions(generate.getGenerators(), replaceMap); + Plan newGenerate = generate.withGenerators(newGenerators).recomputeLogicalProperties(); + newGenerate.getOutputSet().forEach(o -> replaceMap.put(o.getExprId(), o)); + return newGenerate; } @Override - public Plan visitLogicalJoin(LogicalJoin join, Void context) { - join = (LogicalJoin) super.visit(join, context); - Map exprIdSlotMap = collectChildrenOutputMap(join); - List hashConjuncts = updateExpressions(join.getHashJoinConjuncts(), exprIdSlotMap); - // because other join compute on join's output on be, so we need to change slot to join's output - exprIdSlotMap = join.getOutputSet().stream() - .collect(Collectors.toMap(NamedExpression::getExprId, s -> s)); - List otherConjuncts = updateExpressions(join.getOtherJoinConjuncts(), exprIdSlotMap); + public Plan visitLogicalJoin(LogicalJoin join, Map replaceMap) { + join = (LogicalJoin) super.visit(join, replaceMap); + List hashConjuncts = updateExpressions(join.getHashJoinConjuncts(), replaceMap); + join.getOutputSet().forEach(o -> replaceMap.put(o.getExprId(), o)); + List otherConjuncts = updateExpressions(join.getOtherJoinConjuncts(), replaceMap); return join.withJoinConjuncts(hashConjuncts, otherConjuncts).recomputeLogicalProperties(); } @Override - public Plan visitLogicalProject(LogicalProject project, Void context) { - project = (LogicalProject) super.visit(project, context); - Map exprIdSlotMap = collectChildrenOutputMap(project); - List newProjects = updateExpressions(project.getProjects(), exprIdSlotMap); + public Plan visitLogicalProject(LogicalProject project, Map replaceMap) { + project = (LogicalProject) super.visit(project, replaceMap); + List newProjects = updateExpressions(project.getProjects(), replaceMap); + newProjects.forEach(p -> replaceMap.put(p.getExprId(), p.toSlot())); return project.withProjects(newProjects); } @Override - public Plan visitLogicalRepeat(LogicalRepeat repeat, Void context) { - repeat = (LogicalRepeat) super.visit(repeat, context); - Map exprIdSlotMap = collectChildrenOutputMap(repeat); + public Plan visitLogicalRepeat(LogicalRepeat repeat, Map replaceMap) { + repeat = (LogicalRepeat) super.visit(repeat, replaceMap); Set flattenGroupingSetExpr = ImmutableSet.copyOf( ExpressionUtils.flatExpressions(repeat.getGroupingSets())); List newOutputs = Lists.newArrayList(); @@ -128,15 +132,16 @@ public Plan visitLogicalRepeat(LogicalRepeat repeat, Void contex if (flattenGroupingSetExpr.contains(output)) { newOutputs.add(output); } else { - newOutputs.add(updateExpression(output, exprIdSlotMap)); + newOutputs.add(updateExpression(output, replaceMap)); } } + newOutputs.forEach(o -> replaceMap.put(o.getExprId(), o.toSlot())); return repeat.withGroupSetsAndOutput(repeat.getGroupingSets(), newOutputs).recomputeLogicalProperties(); } @Override - public Plan visitLogicalSetOperation(LogicalSetOperation setOperation, Void context) { - setOperation = (LogicalSetOperation) super.visit(setOperation, context); + public Plan visitLogicalSetOperation(LogicalSetOperation setOperation, Map replaceMap) { + setOperation = (LogicalSetOperation) super.visit(setOperation, replaceMap); if (setOperation.children().isEmpty()) { return setOperation; } @@ -150,6 +155,16 @@ public Plan visitLogicalSetOperation(LogicalSetOperation setOperation, Void cont } } } + if (setOperation instanceof LogicalUnion) { + LogicalUnion logicalUnion = (LogicalUnion) setOperation; + for (List constantExprs : logicalUnion.getConstantExprsList()) { + for (int j = 0; j < constantExprs.size(); j++) { + if (constantExprs.get(j).nullable()) { + inputNullable.set(j, true); + } + } + } + } List outputs = setOperation.getOutputs(); List newOutputs = Lists.newArrayListWithCapacity(outputs.size()); for (int i = 0; i < inputNullable.size(); i++) { @@ -160,48 +175,71 @@ public Plan visitLogicalSetOperation(LogicalSetOperation setOperation, Void cont } newOutputs.add(ne instanceof Alias ? (NamedExpression) ne.withChildren(slot) : slot); } + newOutputs.forEach(o -> replaceMap.put(o.getExprId(), o.toSlot())); return setOperation.withNewOutputs(newOutputs).recomputeLogicalProperties(); } @Override - public Plan visitLogicalSort(LogicalSort sort, Void context) { - sort = (LogicalSort) super.visit(sort, context); - Map exprIdSlotMap = collectChildrenOutputMap(sort); + public Plan visitLogicalSort(LogicalSort sort, Map replaceMap) { + sort = (LogicalSort) super.visit(sort, replaceMap); List newKeys = sort.getOrderKeys().stream() - .map(old -> old.withExpression(updateExpression(old.getExpr(), exprIdSlotMap))) + .map(old -> old.withExpression(updateExpression(old.getExpr(), replaceMap))) .collect(ImmutableList.toImmutableList()); return sort.withOrderKeys(newKeys).recomputeLogicalProperties(); } @Override - public Plan visitLogicalTopN(LogicalTopN topN, Void context) { - topN = (LogicalTopN) super.visit(topN, context); - Map exprIdSlotMap = collectChildrenOutputMap(topN); + public Plan visitLogicalTopN(LogicalTopN topN, Map replaceMap) { + topN = (LogicalTopN) super.visit(topN, replaceMap); List newKeys = topN.getOrderKeys().stream() - .map(old -> old.withExpression(updateExpression(old.getExpr(), exprIdSlotMap))) + .map(old -> old.withExpression(updateExpression(old.getExpr(), replaceMap))) .collect(ImmutableList.toImmutableList()); return topN.withOrderKeys(newKeys).recomputeLogicalProperties(); } @Override - public Plan visitLogicalWindow(LogicalWindow window, Void context) { - window = (LogicalWindow) super.visit(window, context); - Map exprIdSlotMap = collectChildrenOutputMap(window); + public Plan visitLogicalWindow(LogicalWindow window, Map replaceMap) { + window = (LogicalWindow) super.visit(window, replaceMap); List windowExpressions = - updateExpressions(window.getWindowExpressions(), exprIdSlotMap); + updateExpressions(window.getWindowExpressions(), replaceMap); + windowExpressions.forEach(w -> replaceMap.put(w.getExprId(), w.toSlot())); return window.withExpression(windowExpressions, window.child()); } - private T updateExpression(T input, Map exprIdSlotMap) { - return (T) input.rewriteDownShortCircuit(e -> e.accept(SlotReferenceReplacer.INSTANCE, exprIdSlotMap)); + @Override + public Plan visitLogicalPartitionTopN(LogicalPartitionTopN partitionTopN, + Map replaceMap) { + partitionTopN = (LogicalPartitionTopN) super.visit(partitionTopN, replaceMap); + List partitionKeys = updateExpressions(partitionTopN.getPartitionKeys(), replaceMap); + List orderKeys = updateExpressions(partitionTopN.getOrderKeys(), replaceMap); + return partitionTopN.withPartitionKeysAndOrderKeys(partitionKeys, orderKeys); + } + + @Override + public Plan visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, Map replaceMap) { + Map consumerToProducerOutputMap = new LinkedHashMap<>(); + Map producerToConsumerOutputMap = new LinkedHashMap<>(); + for (Slot producerOutputSlot : cteConsumer.getConsumerToProducerOutputMap().values()) { + Slot newProducerOutputSlot = updateExpression(producerOutputSlot, replaceMap); + Slot newConsumerOutputSlot = cteConsumer.getProducerToConsumerOutputMap().get(producerOutputSlot) + .withNullable(newProducerOutputSlot.nullable()); + producerToConsumerOutputMap.put(newProducerOutputSlot, newConsumerOutputSlot); + consumerToProducerOutputMap.put(newConsumerOutputSlot, newProducerOutputSlot); + replaceMap.put(newConsumerOutputSlot.getExprId(), newConsumerOutputSlot); + } + return cteConsumer.withTwoMaps(consumerToProducerOutputMap, producerToConsumerOutputMap); + } + + private T updateExpression(T input, Map replaceMap) { + return (T) input.rewriteDownShortCircuit(e -> e.accept(SlotReferenceReplacer.INSTANCE, replaceMap)); } - private List updateExpressions(List inputs, Map exprIdSlotMap) { - return inputs.stream().map(i -> updateExpression(i, exprIdSlotMap)).collect(ImmutableList.toImmutableList()); + private List updateExpressions(List inputs, Map replaceMap) { + return inputs.stream().map(i -> updateExpression(i, replaceMap)).collect(ImmutableList.toImmutableList()); } - private Set updateExpressions(Set inputs, Map exprIdSlotMap) { - return inputs.stream().map(i -> updateExpression(i, exprIdSlotMap)).collect(ImmutableSet.toImmutableSet()); + private Set updateExpressions(Set inputs, Map replaceMap) { + return inputs.stream().map(i -> updateExpression(i, replaceMap)).collect(ImmutableSet.toImmutableSet()); } private Map collectChildrenOutputMap(LogicalPlan plan) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AggScalarSubQueryToWindowFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AggScalarSubQueryToWindowFunction.java index d0e57424874b48..9f588d4c5dac50 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AggScalarSubQueryToWindowFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AggScalarSubQueryToWindowFunction.java @@ -32,6 +32,7 @@ import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionVisitor; import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalApply; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; @@ -248,11 +249,11 @@ private boolean checkProject() { * 3. the remaining table in step 2 should be correlated table for inner plan */ private boolean checkRelation(List correlatedSlots) { - List outerTables = outerPlans.stream().filter(LogicalRelation.class::isInstance) - .map(LogicalRelation.class::cast) + List outerTables = outerPlans.stream().filter(CatalogRelation.class::isInstance) + .map(CatalogRelation.class::cast) .collect(Collectors.toList()); - List innerTables = innerPlans.stream().filter(LogicalRelation.class::isInstance) - .map(LogicalRelation.class::cast) + List innerTables = innerPlans.stream().filter(CatalogRelation.class::isInstance) + .map(CatalogRelation.class::cast) .collect(Collectors.toList()); List outerIds = outerTables.stream().map(node -> node.getTable().getId()).collect(Collectors.toList()); @@ -273,15 +274,16 @@ private boolean checkRelation(List correlatedSlots) { Set correlatedRelationOutput = outerTables.stream() .filter(node -> outerIds.contains(node.getTable().getId())) + .map(LogicalRelation.class::cast) .map(LogicalRelation::getOutputExprIdSet).flatMap(Collection::stream).collect(Collectors.toSet()); return ExpressionUtils.collect(correlatedSlots, NamedExpression.class::isInstance).stream() .map(NamedExpression.class::cast) .allMatch(e -> correlatedRelationOutput.contains(e.getExprId())); } - private void createSlotMapping(List outerTables, List innerTables) { - for (LogicalRelation outerTable : outerTables) { - for (LogicalRelation innerTable : innerTables) { + private void createSlotMapping(List outerTables, List innerTables) { + for (CatalogRelation outerTable : outerTables) { + for (CatalogRelation innerTable : innerTables) { if (innerTable.getTable().getId() == outerTable.getTable().getId()) { for (Slot innerSlot : innerTable.getOutput()) { for (Slot outerSlot : outerTable.getOutput()) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/BuildCTEAnchorAndCTEProducer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/BuildCTEAnchorAndCTEProducer.java deleted file mode 100644 index 11700ecb7785c3..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/BuildCTEAnchorAndCTEProducer.java +++ /dev/null @@ -1,67 +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.CascadesContext; -import org.apache.doris.nereids.rules.Rule; -import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.expressions.CTEId; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; -import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; -import org.apache.doris.qe.ConnectContext; - -/** - * BuildCTEAnchorAndCTEProducer. - */ -public class BuildCTEAnchorAndCTEProducer extends OneRewriteRuleFactory { - - @Override - public Rule build() { - return logicalCTE().thenApply(ctx -> { - return rewrite(ctx.root, ctx.cascadesContext); - }).toRule(RuleType.BUILD_CTE_ANCHOR_AND_CTE_PRODUCER); - } - - @SuppressWarnings({"unchecked", "rawtypes"}) - private LogicalPlan rewrite(LogicalPlan p, CascadesContext cascadesContext) { - if (!(p instanceof LogicalCTE)) { - return p; - } - LogicalCTE logicalCTE = (LogicalCTE) p; - LogicalPlan child = (LogicalPlan) logicalCTE.child(); - if (!(child instanceof LogicalEmptyRelation)) { - for (int i = logicalCTE.getAliasQueries().size() - 1; i >= 0; i--) { - LogicalSubQueryAlias s = (LogicalSubQueryAlias) logicalCTE.getAliasQueries().get(i); - CTEId id = logicalCTE.findCTEId(s.getAlias()); - if (cascadesContext.cteReferencedCount(id) - <= ConnectContext.get().getSessionVariable().inlineCTEReferencedThreshold - || !ConnectContext.get().getSessionVariable().getEnablePipelineEngine()) { - continue; - } - LogicalCTEProducer logicalCTEProducer = new LogicalCTEProducer( - rewrite((LogicalPlan) s.child(), cascadesContext), id); - child = new LogicalCTEAnchor(logicalCTEProducer, child, id); - } - } - return child; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEInline.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEInline.java new file mode 100644 index 00000000000000..46216ec0438290 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEInline.java @@ -0,0 +1,112 @@ +// 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.jobs.JobContext; +import org.apache.doris.nereids.trees.copier.DeepCopierContext; +import org.apache.doris.nereids.trees.copier.LogicalPlanDeepCopier; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalProject; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * pull up LogicalCteAnchor to the top of plan to avoid CteAnchor break other rewrite rules pattern + * The front producer may depend on the back producer in {@code List>} + * After this rule, we normalize all CteAnchor in plan, all CteAnchor under CteProducer should pull out + * and put all of them to the top of plan depends on dependency tree of them. + */ +public class CTEInline extends DefaultPlanRewriter> implements CustomRewriter { + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + Plan root = plan.accept(this, null); + // collect cte id to consumer + root.foreach(p -> { + if (p instanceof LogicalCTEConsumer) { + jobContext.getCascadesContext().putCTEIdToConsumer(((LogicalCTEConsumer) p)); + } + }); + return root; + } + + @Override + public Plan visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, + LogicalCTEProducer producer) { + if (producer != null) { + // process upper anchor + List children = cteAnchor.children().stream() + .map(c -> c.accept(this, producer)) + .collect(ImmutableList.toImmutableList()); + return cteAnchor.withChildren(children); + } else { + // process this anchor + List consumers = cteAnchor.child(1).collectToList(p -> { + if (p instanceof LogicalCTEConsumer) { + return ((LogicalCTEConsumer) p).getCteId().equals(cteAnchor.getCteId()); + } + return false; + }); + if (ConnectContext.get().getSessionVariable().getEnablePipelineEngine() + && ConnectContext.get().getSessionVariable().enableCTEMaterialize + && consumers.size() > ConnectContext.get().getSessionVariable().inlineCTEReferencedThreshold) { + // not inline + Plan right = cteAnchor.right().accept(this, null); + return cteAnchor.withChildren(cteAnchor.left(), right); + } else { + // should inline + Plan root = cteAnchor.right().accept(this, (LogicalCTEProducer) cteAnchor.left()); + // process child + return root.accept(this, null); + } + } + } + + @Override + public Plan visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, LogicalCTEProducer producer) { + if (producer != null && cteConsumer.getCteId().equals(producer.getCteId())) { + DeepCopierContext deepCopierContext = new DeepCopierContext(); + Plan inlinedPlan = LogicalPlanDeepCopier.INSTANCE + .deepCopy((LogicalPlan) producer.child(), deepCopierContext); + List projects = Lists.newArrayList(); + for (Slot consumerSlot : cteConsumer.getOutput()) { + Slot producerSlot = cteConsumer.getProducerSlot(consumerSlot); + ExprId inlineExprId = deepCopierContext.exprIdReplaceMap.get(producerSlot.getExprId()); + Alias alias = new Alias(consumerSlot.getExprId(), producerSlot.withExprId(inlineExprId), + consumerSlot.getName()); + projects.add(alias); + } + return new LogicalProject<>(projects, inlinedPlan); + } + return cteConsumer; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEProducerRewrite.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEProducerRewrite.java deleted file mode 100644 index ee197c429c8552..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CTEProducerRewrite.java +++ /dev/null @@ -1,122 +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.CascadesContext; -import org.apache.doris.nereids.jobs.executor.Rewriter; -import org.apache.doris.nereids.rules.Rule; -import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.expressions.CTEId; -import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; -import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -import org.apache.doris.nereids.util.ExpressionUtils; - -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableSet; -import org.apache.commons.collections.CollectionUtils; - -import java.util.ArrayList; -import java.util.HashSet; -import java.util.Map.Entry; -import java.util.Set; -import java.util.stream.Collectors; - -/** - * Rewrite CTE Producer recursively. - */ -public class CTEProducerRewrite extends OneRewriteRuleFactory { - - @Override - public Rule build() { - return logicalCTEProducer().when(p -> !p.isRewritten()).thenApply(ctx -> { - LogicalCTEProducer cteProducer = ctx.root; - Set projects = ctx.cascadesContext.getProjectForProducer(cteProducer.getCteId()); - LogicalPlan child = tryToConstructFilter(ctx.cascadesContext, cteProducer.getCteId(), - (LogicalPlan) ctx.root.child()); - if (CollectionUtils.isNotEmpty(projects) - && ctx.cascadesContext.couldPruneColumnOnProducer(cteProducer.getCteId())) { - child = new LogicalProject(ImmutableList.copyOf(projects), child); - } - CascadesContext rewrittenCtx = ctx.cascadesContext.forkForCTEProducer(child); - Rewriter rewriter = new Rewriter(rewrittenCtx); - rewriter.execute(); - return cteProducer.withChildrenAndProjects(ImmutableList.of(rewrittenCtx.getRewritePlan()), - new ArrayList<>(child.getOutput()), true); - }).toRule(RuleType.CTE_PRODUCER_REWRITE); - } - - /* - * An expression can only be pushed down if it has filter expressions on all consumers that reference the slot. - * For example, let's assume a producer has two consumers, consumer1 and consumer2: - * - * filter(a > 5 and b < 1) -> consumer1 - * filter(a < 8) -> consumer2 - * - * In this case, the only expression that can be pushed down to the producer is filter(a > 5 or a < 8). - */ - private LogicalPlan tryToConstructFilter(CascadesContext cascadesContext, CTEId cteId, LogicalPlan child) { - Set consumerIds = cascadesContext.getCteIdToConsumers().get(cteId).stream() - .map(LogicalCTEConsumer::getConsumerId) - .collect(Collectors.toSet()); - Set> filtersAboveEachConsumer = cascadesContext.getConsumerIdToFilters().entrySet().stream() - .filter(kv -> consumerIds.contains(kv.getKey())) - .map(Entry::getValue) - .collect(Collectors.toSet()); - Set someone = filtersAboveEachConsumer.stream().findFirst().orElse(null); - if (someone == null) { - return child; - } - int filterSize = cascadesContext.getCteIdToConsumers().get(cteId).size(); - Set filter = new HashSet<>(); - for (Expression f : someone) { - int matchCount = 1; - Set slots = f.collect(e -> e instanceof SlotReference); - Set mightBeJoined = new HashSet<>(); - for (Set another : filtersAboveEachConsumer) { - if (another.equals(someone)) { - continue; - } - Set matched = new HashSet<>(); - for (Expression e : another) { - Set otherSlots = e.collect(ae -> ae instanceof SlotReference); - if (otherSlots.equals(slots)) { - matched.add(e); - } - } - if (!matched.isEmpty()) { - matchCount++; - } - mightBeJoined.addAll(matched); - } - if (matchCount >= filterSize) { - mightBeJoined.add(f); - filter.add(ExpressionUtils.or(mightBeJoined)); - } - } - if (!filter.isEmpty()) { - return new LogicalFilter(ImmutableSet.of(ExpressionUtils.and(filter)), child); - } - return child; - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckMatchExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckMatchExpression.java index 3b04059a488081..dbfe480baff444 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckMatchExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CheckMatchExpression.java @@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Match; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import java.util.List; @@ -41,15 +42,15 @@ public Rule build() { .toRule(RuleType.CHECK_MATCH_EXPRESSION); } - private LogicalFilter checkChildren(LogicalFilter filter) { + private Plan checkChildren(LogicalFilter filter) { List expressions = filter.getExpressions(); for (Expression expr : expressions) { if (expr instanceof Match) { Match matchExpression = (Match) expr; if (!(matchExpression.left() instanceof SlotReference) || !(matchExpression.right() instanceof Literal)) { - throw new AnalysisException(String.format( - "Only support match left operand is SlotRef, right operand is Literal")); + throw new AnalysisException(String.format("Only support match left operand is SlotRef," + + " right operand is Literal. But meet expression %s", matchExpression)); } } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectFilterAboveConsumer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectFilterAboveConsumer.java index 6646a59c2a1eb8..4c4249b69c7c39 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectFilterAboveConsumer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectFilterAboveConsumer.java @@ -40,11 +40,11 @@ public Rule build() { for (Expression expr : exprs) { Expression rewrittenExpr = expr.rewriteUp(e -> { if (e instanceof Slot) { - return cteConsumer.findProducerSlot((Slot) e); + return cteConsumer.getProducerSlot((Slot) e); } return e; }); - ctx.cascadesContext.putConsumerIdToFilter(cteConsumer.getConsumerId(), rewrittenExpr); + ctx.cascadesContext.putConsumerIdToFilter(cteConsumer.getRelationId(), rewrittenExpr); } return ctx.root; }).toRule(RuleType.COLLECT_FILTER_ON_CONSUMER); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectProjectAboveConsumer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectProjectAboveConsumer.java index e7f85ca79e90c7..43f8231cbc3d39 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectProjectAboveConsumer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/CollectProjectAboveConsumer.java @@ -48,8 +48,8 @@ public List buildRules() { collectProject(ctx.cascadesContext, namedExpressions, cteConsumer); return ctx.root; })), - RuleType.COLLECT_PROJECT_ABOVE_FILTER_CONSUMER.build(logicalProject(logicalFilter(logicalCTEConsumer())) - .thenApply(ctx -> { + RuleType.COLLECT_PROJECT_ABOVE_FILTER_CONSUMER + .build(logicalProject(logicalFilter(logicalCTEConsumer())).thenApply(ctx -> { LogicalProject> project = ctx.root; LogicalFilter filter = project.child(); Set filterSlots = filter.getInputSlots(); @@ -72,7 +72,7 @@ private static void collectProject(CascadesContext ctx, if (!(node instanceof Slot)) { return; } - Slot slot = cteConsumer.findProducerSlot((Slot) node); + Slot slot = cteConsumer.getProducerSlot((Slot) node); ctx.putCTEIdToProject(cteConsumer.getCteId(), slot); ctx.markConsumerUnderProject(cteConsumer); }); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java index 7f3205d8f2cc06..2fc17a717d06f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalIntersect; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; @@ -161,10 +162,15 @@ public Plan visitLogicalIntersect(LogicalIntersect intersect, PruneContext conte } @Override - public Plan visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, PruneContext context) { + public Plan visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, PruneContext context) { return skipPruneThisAndFirstLevelChildren(olapTableSink); } + @Override + public Plan visitLogicalFileSink(LogicalFileSink fileSink, PruneContext context) { + return skipPruneThisAndFirstLevelChildren(fileSink); + } + // the backend not support filter(project(agg)), so we can not prune the key set in the agg, // only prune the agg functions here @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateFilter.java index e3d3fb48e1fd68..5f5d8b39a83a7c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateFilter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateFilter.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; @@ -36,11 +37,13 @@ public class EliminateFilter extends OneRewriteRuleFactory { public Rule build() { return logicalFilter() .when(filter -> filter.getConjuncts().stream().anyMatch(BooleanLiteral.class::isInstance)) - .then(filter -> { + .thenApply(ctx -> { + LogicalFilter filter = ctx.root; Set newConjuncts = Sets.newHashSetWithExpectedSize(filter.getConjuncts().size()); for (Expression expression : filter.getConjuncts()) { if (expression == BooleanLiteral.FALSE) { - return new LogicalEmptyRelation(filter.getOutput()); + return new LogicalEmptyRelation(ctx.statementContext.getNextRelationId(), + filter.getOutput()); } else if (expression != BooleanLiteral.TRUE) { newConjuncts.add(expression); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateLimit.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateLimit.java index 01900862563a40..9cc19e47d8b5c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateLimit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateLimit.java @@ -29,7 +29,8 @@ public class EliminateLimit extends OneRewriteRuleFactory { public Rule build() { return logicalLimit() .when(limit -> limit.getLimit() == 0) - .then(limit -> new LogicalEmptyRelation(limit.getOutput())) + .thenApply(ctx -> new LogicalEmptyRelation(ctx.statementContext.getNextRelationId(), + ctx.root.getOutput())) .toRule(RuleType.ELIMINATE_LIMIT); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateUnnecessaryProject.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateUnnecessaryProject.java index 9b0a0de52286eb..adb355f94dbe28 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateUnnecessaryProject.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/EliminateUnnecessaryProject.java @@ -19,6 +19,7 @@ import org.apache.doris.nereids.annotation.DependsRules; import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; @@ -56,7 +57,7 @@ private Plan rewrite(Plan plan, boolean outputSavePoint) { private Plan rewriteProject(LogicalProject project, boolean outputSavePoint) { if (project.child() instanceof LogicalEmptyRelation) { // eliminate unnecessary project - return new LogicalEmptyRelation(project.getProjects()); + return new LogicalEmptyRelation(StatementScopeIdGenerator.newRelationId(), project.getProjects()); } else if (project.canEliminate() && outputSavePoint && project.getOutputSet().equals(project.child().getOutputSet())) { // eliminate unnecessary project diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InApplyToJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InApplyToJoin.java index 261316e83ed9aa..64d8defcfe848a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InApplyToJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InApplyToJoin.java @@ -34,6 +34,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalApply; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.util.ExpressionUtils; import com.google.common.collect.ImmutableList; @@ -93,7 +94,9 @@ select t1.k1 from bigtable t1 left semi join (select bitmap_union(k2) x from bit //in-predicate to equal Expression predicate; Expression left = ((InSubquery) apply.getSubqueryExpr()).getCompareExpr(); - Expression right = apply.getSubqueryExpr().getSubqueryOutput(); + // TODO: trick here, because when deep copy logical plan the apply right child + // is not same with query plan in subquery expr, since the scan node copy twice + Expression right = apply.getSubqueryExpr().getSubqueryOutput((LogicalPlan) apply.right()); if (apply.isCorrelated()) { predicate = ExpressionUtils.and(new EqualTo(left, right), apply.getCorrelationFilter().get()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InlineCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InlineCTE.java deleted file mode 100644 index a50f42a2c07c0d..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/InlineCTE.java +++ /dev/null @@ -1,73 +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.trees.expressions.Alias; -import org.apache.doris.nereids.trees.expressions.NamedExpression; -import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; -import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -import org.apache.doris.qe.ConnectContext; - -import java.util.ArrayList; -import java.util.List; - -/** - * A CTEConsumer would be converted to a inlined plan if corresponding CTE referenced less than or - * equal inline_cte_referenced_threshold (it's a session variable, by default is 1). - */ -public class InlineCTE extends OneRewriteRuleFactory { - - private static final int INLINE_CTE_REFERENCED_THRESHOLD = 1; - - @Override - public Rule build() { - return logicalCTEConsumer().thenApply(ctx -> { - LogicalCTEConsumer cteConsumer = ctx.root; - int refCount = ctx.cascadesContext.cteReferencedCount(cteConsumer.getCteId()); - /* - * Current we only implement CTE Materialize on pipeline engine and only materialize those CTE whose - * refCount > NereidsRewriter.INLINE_CTE_REFERENCED_THRESHOLD. - */ - if (ConnectContext.get().getSessionVariable().getEnablePipelineEngine() - && ConnectContext.get().getSessionVariable().enableCTEMaterialize - && refCount > INLINE_CTE_REFERENCED_THRESHOLD) { - return cteConsumer; - } - LogicalPlan inlinedPlan = ctx.cascadesContext.findCTEPlanForInline(cteConsumer.getCteId()); - List inlinedPlanOutput = inlinedPlan.getOutput(); - List cteConsumerOutput = cteConsumer.getOutput(); - List projects = new ArrayList<>(); - for (Slot inlineSlot : inlinedPlanOutput) { - String name = inlineSlot.getName(); - for (Slot consumerSlot : cteConsumerOutput) { - if (consumerSlot.getName().equals(name)) { - Alias alias = new Alias(consumerSlot.getExprId(), inlineSlot, name); - projects.add(alias); - break; - } - } - } - return new LogicalProject<>(projects, - inlinedPlan); - }).toRule(RuleType.INLINE_CTE); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java new file mode 100644 index 00000000000000..027a20b3db43ca --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java @@ -0,0 +1,90 @@ +// 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.jobs.JobContext; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; + +import com.google.common.collect.Lists; + +import java.util.List; + +/** + * pull up LogicalCteAnchor to the top of plan to avoid CteAnchor break other rewrite rules pattern + * The front producer may depend on the back producer in {@code List>} + * After this rule, we normalize all CteAnchor in plan, all CteAnchor under CteProducer should pull out + * and put all of them to the top of plan depends on dependency tree of them. + */ +public class PullUpCteAnchor extends DefaultPlanRewriter>> implements CustomRewriter { + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + List> producers = Lists.newArrayList(); + return rewriteRoot(plan, producers); + } + + private Plan rewriteRoot(Plan plan, List> producers) { + Plan root = plan.accept(this, producers); + for (LogicalCTEProducer producer : producers) { + root = new LogicalCTEAnchor<>(producer.getCteId(), producer, root); + } + return root; + } + + @Override + public Plan visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, + List> producers) { + // 1. process child side + Plan root = cteAnchor.child(1).accept(this, producers); + // 2. process producers side, need to collect all producer + cteAnchor.child(0).accept(this, producers); + return root; + } + + @Override + public LogicalCTEProducer visitLogicalCTEProducer(LogicalCTEProducer cteProducer, + List> producers) { + List> childProducers = Lists.newArrayList(); + Plan child = cteProducer.child().accept(this, childProducers); + LogicalCTEProducer newProducer = (LogicalCTEProducer) cteProducer.withChildren(child); + // because current producer relay on it child's producers, so add current producer first. + producers.add(newProducer); + producers.addAll(childProducers); + return newProducer; + } + + // we should keep that sink node is the top node of the plan tree. + // currently, it's one of the olap table sink and file sink. + @Override + public Plan visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, + List> producers) { + return olapTableSink.withChildren(rewriteRoot(olapTableSink.child(), producers)); + } + + @Override + public Plan visitLogicalFileSink(LogicalFileSink fileSink, + List> producers) { + return fileSink.withChildren(rewriteRoot(fileSink.child(), producers)); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughAggregation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughAggregation.java index e1f445706791dc..217834c4bdc518 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughAggregation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughAggregation.java @@ -83,9 +83,12 @@ public Rule build() { */ public static Set getCanPushDownSlots(LogicalAggregate aggregate) { Set canPushDownSlots = new HashSet<>(); - if (aggregate.hasRepeat()) { + if (aggregate.getSourceRepeat().isPresent()) { // When there is a repeat, the push-down condition is consistent with the repeat - canPushDownSlots.addAll(aggregate.getSourceRepeat().get().getCommonGroupingSetExpressions()); + aggregate.getSourceRepeat().get().getCommonGroupingSetExpressions().stream() + .filter(Slot.class::isInstance) + .map(Slot.class::cast) + .forEach(canPushDownSlots::add); } else { for (Expression groupByExpression : aggregate.getGroupByExpressions()) { if (groupByExpression instanceof Slot) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughCTE.java deleted file mode 100644 index 47dc5468bc05a5..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughCTE.java +++ /dev/null @@ -1,39 +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.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; -import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; - -/** - * Push filter through CTE. - */ -public class PushdownFilterThroughCTE extends OneRewriteRuleFactory { - - @Override - public Rule build() { - return logicalFilter(logicalCTE()).thenApply(ctx -> { - LogicalFilter> filter = ctx.root; - LogicalCTE anchor = filter.child(); - return anchor.withChildren(filter.withChildren(anchor.child())); - }).toRule(RuleType.PUSHDOWN_FILTER_THROUGH_CTE); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughCTEAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughCTEAnchor.java deleted file mode 100644 index 2942e3654eb9c2..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughCTEAnchor.java +++ /dev/null @@ -1,39 +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.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; -import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; - -/** - * Push filter through CTEAnchor. - */ -public class PushdownFilterThroughCTEAnchor extends OneRewriteRuleFactory { - - @Override - public Rule build() { - return logicalFilter(logicalCTEAnchor()).thenApply(ctx -> { - LogicalFilter> filter = ctx.root; - LogicalCTEAnchor anchor = filter.child(); - return anchor.withChildren(anchor.left(), filter.withChildren((Plan) anchor.right())); - }).toRule(RuleType.PUSHDOWN_FILTER_THROUGH_CTE_ANCHOR); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughWindow.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughWindow.java index 61ba62f8ac2a7e..0addaeac0a0da0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughWindow.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughWindow.java @@ -31,6 +31,7 @@ import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; @@ -77,7 +78,8 @@ public class PushdownFilterThroughWindow extends OneRewriteRuleFactory { @Override public Rule build() { - return logicalFilter(logicalWindow()).then(filter -> { + return logicalFilter(logicalWindow()).thenApply(ctx -> { + LogicalFilter> filter = ctx.root; LogicalWindow window = filter.child(); // We have already done such optimization rule, so just ignore it. @@ -117,7 +119,7 @@ public Rule build() { limitVal--; } if (limitVal < 0) { - return new LogicalEmptyRelation(filter.getOutput()); + return new LogicalEmptyRelation(ctx.statementContext.getNextRelationId(), filter.getOutput()); } if (hasPartitionLimit) { partitionLimit = Math.min(partitionLimit, limitVal); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownLimit.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownLimit.java index 8bc140467d7622..48e2665121d37b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownLimit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownLimit.java @@ -20,6 +20,7 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.UnaryNode; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.Limit; import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier; @@ -134,7 +135,8 @@ public List buildRules() { }).toRule(RuleType.PUSH_LIMIT_INTO_SORT), logicalLimit(logicalOneRowRelation()) .then(limit -> limit.getLimit() > 0 && limit.getOffset() == 0 - ? limit.child() : new LogicalEmptyRelation(limit.child().getOutput())) + ? limit.child() : new LogicalEmptyRelation(StatementScopeIdGenerator.newRelationId(), + limit.child().getOutput())) .toRule(RuleType.ELIMINATE_LIMIT_ON_ONE_ROW_RELATION), logicalLimit(logicalEmptyRelation()) .then(UnaryNode::child) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownProjectThroughCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownProjectThroughCTE.java deleted file mode 100644 index 9db0b5adc4dbb4..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownProjectThroughCTE.java +++ /dev/null @@ -1,39 +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.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTE; -import org.apache.doris.nereids.trees.plans.logical.LogicalProject; - -/** - * Push project through CTE. - */ -public class PushdownProjectThroughCTE extends OneRewriteRuleFactory { - - @Override - public Rule build() { - return logicalProject(logicalCTE()).thenApply(ctx -> { - LogicalProject> project = ctx.root; - LogicalCTE anchor = project.child(); - return anchor.withChildren(project.withChildren(anchor.child())); - }).toRule(RuleType.PUSH_DOWN_PROJECT_THROUGH_CTE); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownProjectThroughCTEAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownProjectThroughCTEAnchor.java deleted file mode 100644 index d92499043dc822..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PushdownProjectThroughCTEAnchor.java +++ /dev/null @@ -1,39 +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.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; -import org.apache.doris.nereids.trees.plans.logical.LogicalProject; - -/** - * Push project through CTEAnchor. - */ -public class PushdownProjectThroughCTEAnchor extends OneRewriteRuleFactory { - - @Override - public Rule build() { - return logicalProject(logicalCTEAnchor()).thenApply(ctx -> { - LogicalProject> project = ctx.root; - LogicalCTEAnchor anchor = project.child(); - return anchor.withChildren(anchor.child(0), project.withChildren(anchor.child(1))); - }).toRule(RuleType.PUSH_DOWN_PROJECT_THROUGH_CTE_ANCHOR); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ReorderJoin.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ReorderJoin.java index 5e14405191b805..a9e685b1c89219 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ReorderJoin.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ReorderJoin.java @@ -87,7 +87,7 @@ public Rule build() { Plan plan = joinToMultiJoin(filter, planToHintType); Preconditions.checkState(plan instanceof MultiJoin); MultiJoin multiJoin = (MultiJoin) plan; - ctx.statementContext.setMaxNArayInnerJoin(multiJoin.children().size()); + ctx.statementContext.setMaxNAryInnerJoin(multiJoin.children().size()); Plan after = multiJoinToJoin(multiJoin, planToHintType); return after; }).toRule(RuleType.REORDER_JOIN); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/RewriteCteChildren.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/RewriteCteChildren.java new file mode 100644 index 00000000000000..d88ef62e314eaf --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/RewriteCteChildren.java @@ -0,0 +1,189 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.rewrite; + +import org.apache.doris.nereids.CascadesContext; +import org.apache.doris.nereids.annotation.DependsRules; +import org.apache.doris.nereids.jobs.JobContext; +import org.apache.doris.nereids.jobs.executor.Rewriter; +import org.apache.doris.nereids.jobs.rewrite.RewriteJob; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.CTEId; +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.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +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.visitor.CustomRewriter; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; +import org.apache.doris.nereids.util.ExpressionUtils; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import org.apache.commons.collections.CollectionUtils; + +import java.util.HashSet; +import java.util.List; +import java.util.Map.Entry; +import java.util.Optional; +import java.util.Set; +import java.util.stream.Collectors; + +/** + * rewrite CteAnchor consumer side and producer side recursively, all CteAnchor must at top of the plan + */ +@DependsRules({PullUpCteAnchor.class, CTEInline.class}) +public class RewriteCteChildren extends DefaultPlanRewriter implements CustomRewriter { + + private final List jobs; + + public RewriteCteChildren(List jobs) { + this.jobs = jobs; + } + + @Override + public Plan rewriteRoot(Plan plan, JobContext jobContext) { + return plan.accept(this, jobContext.getCascadesContext()); + } + + @Override + public Plan visit(Plan plan, CascadesContext context) { + Rewriter.getCteChildrenRewriter(context, jobs).execute(); + return context.getRewritePlan(); + } + + @Override + public Plan visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, + CascadesContext cascadesContext) { + LogicalPlan outer; + if (cascadesContext.getStatementContext().getRewrittenCtePlan().containsKey(null)) { + outer = cascadesContext.getStatementContext().getRewrittenCtePlan().get(null); + } else { + CascadesContext outerCascadesCtx = CascadesContext.newSubtreeContext( + Optional.empty(), cascadesContext, cteAnchor.child(1), + cascadesContext.getCurrentJobContext().getRequiredProperties()); + outer = (LogicalPlan) cteAnchor.child(1).accept(this, outerCascadesCtx); + cascadesContext.getStatementContext().getRewrittenCtePlan().put(null, outer); + } + boolean reserveAnchor = outer.anyMatch(p -> { + if (p instanceof LogicalCTEConsumer) { + LogicalCTEConsumer logicalCTEConsumer = (LogicalCTEConsumer) p; + return logicalCTEConsumer.getCteId().equals(cteAnchor.getCteId()); + } + return false; + }); + if (!reserveAnchor) { + return outer; + } + Plan producer = cteAnchor.child(0).accept(this, cascadesContext); + return cteAnchor.withChildren(producer, outer); + } + + @Override + public Plan visitLogicalCTEProducer(LogicalCTEProducer cteProducer, + CascadesContext cascadesContext) { + LogicalPlan child; + if (cascadesContext.getStatementContext().getRewrittenCtePlan().containsKey(cteProducer.getCteId())) { + child = cascadesContext.getStatementContext().getRewrittenCtePlan().get(cteProducer.getCteId()); + } else { + child = (LogicalPlan) cteProducer.child(); + child = tryToConstructFilter(cascadesContext, cteProducer.getCteId(), child); + Set projects = cascadesContext.getProjectForProducer(cteProducer.getCteId()); + if (CollectionUtils.isNotEmpty(projects) + && cascadesContext.couldPruneColumnOnProducer(cteProducer.getCteId())) { + child = new LogicalProject<>(ImmutableList.copyOf(projects), child); + child = pushPlanUnderAnchor(child); + } + CascadesContext rewrittenCtx = CascadesContext.newSubtreeContext( + Optional.of(cteProducer.getCteId()), cascadesContext, child, PhysicalProperties.ANY); + child = (LogicalPlan) child.accept(this, rewrittenCtx); + cascadesContext.getStatementContext().getRewrittenCtePlan().put(cteProducer.getCteId(), child); + } + return cteProducer.withChildren(child); + } + + private LogicalPlan pushPlanUnderAnchor(LogicalPlan plan) { + if (plan.child(0) instanceof LogicalCTEAnchor) { + LogicalPlan child = (LogicalPlan) plan.withChildren(plan.child(0).child(1)); + return (LogicalPlan) plan.child(0).withChildren( + plan.child(0).child(0), pushPlanUnderAnchor(child)); + } + return plan; + } + + /* + * An expression can only be pushed down if it has filter expressions on all consumers that reference the slot. + * For example, let's assume a producer has two consumers, consumer1 and consumer2: + * + * filter(a > 5 and b < 1) -> consumer1 + * filter(a < 8) -> consumer2 + * + * In this case, the only expression that can be pushed down to the producer is filter(a > 5 or a < 8). + */ + private LogicalPlan tryToConstructFilter(CascadesContext cascadesContext, CTEId cteId, LogicalPlan child) { + Set consumerIds = cascadesContext.getCteIdToConsumers().get(cteId).stream() + .map(LogicalCTEConsumer::getRelationId) + .collect(Collectors.toSet()); + Set> filtersAboveEachConsumer = cascadesContext.getConsumerIdToFilters().entrySet().stream() + .filter(kv -> consumerIds.contains(kv.getKey())) + .map(Entry::getValue) + .collect(Collectors.toSet()); + Set someone = filtersAboveEachConsumer.stream().findFirst().orElse(null); + if (someone == null) { + return child; + } + int filterSize = cascadesContext.getCteIdToConsumers().get(cteId).size(); + Set conjuncts = new HashSet<>(); + for (Expression f : someone) { + int matchCount = 1; + Set slots = f.collect(e -> e instanceof SlotReference); + Set mightBeJoined = new HashSet<>(); + for (Set another : filtersAboveEachConsumer) { + if (another.equals(someone)) { + continue; + } + Set matched = new HashSet<>(); + for (Expression e : another) { + Set otherSlots = e.collect(ae -> ae instanceof SlotReference); + if (otherSlots.equals(slots)) { + matched.add(e); + } + } + if (!matched.isEmpty()) { + matchCount++; + } + mightBeJoined.addAll(matched); + } + if (matchCount >= filterSize) { + mightBeJoined.add(f); + conjuncts.add(ExpressionUtils.or(mightBeJoined)); + } + } + if (!conjuncts.isEmpty()) { + LogicalPlan filter = new LogicalFilter<>(ImmutableSet.of(ExpressionUtils.and(conjuncts)), child); + return pushPlanUnderAnchor(filter); + } + return child; + } +} 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 5fe4b05c4b2462..a7cf8648fea3ce 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 @@ -40,6 +40,7 @@ import org.apache.doris.nereids.trees.expressions.functions.window.Rank; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.algebra.Aggregate; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.nereids.trees.plans.algebra.EmptyRelation; import org.apache.doris.nereids.trees.plans.algebra.Filter; import org.apache.doris.nereids.trees.plans.algebra.Generate; @@ -47,7 +48,6 @@ 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; @@ -215,7 +215,7 @@ public static StatsCalculator estimate(GroupExpression groupExpression, boolean // For unit test only public static void estimate(GroupExpression groupExpression, CascadesContext context) { StatsCalculator statsCalculator = new StatsCalculator(groupExpression, false, - new HashMap<>(), false, Collections.EMPTY_MAP, context); + new HashMap<>(), false, Collections.emptyMap(), context); statsCalculator.estimate(); } @@ -288,18 +288,18 @@ public Statistics visitLogicalFilter(LogicalFilter filter, Void @Override public Statistics visitLogicalOlapScan(LogicalOlapScan olapScan, Void context) { - return computeScan(olapScan); + return computeCatalogRelation(olapScan); } @Override public Statistics visitLogicalSchemaScan(LogicalSchemaScan schemaScan, Void context) { - return computeScan(schemaScan); + return computeCatalogRelation(schemaScan); } @Override public Statistics visitLogicalFileScan(LogicalFileScan fileScan, Void context) { fileScan.getExpressions(); - return computeScan(fileScan); + return computeCatalogRelation(fileScan); } @Override @@ -310,13 +310,13 @@ public Statistics visitLogicalTVFRelation(LogicalTVFRelation tvfRelation, Void c @Override public Statistics visitLogicalJdbcScan(LogicalJdbcScan jdbcScan, Void context) { jdbcScan.getExpressions(); - return computeScan(jdbcScan); + return computeCatalogRelation(jdbcScan); } @Override public Statistics visitLogicalEsScan(LogicalEsScan esScan, Void context) { esScan.getExpressions(); - return computeScan(esScan); + return computeCatalogRelation(esScan); } @Override @@ -420,17 +420,17 @@ public Statistics visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelat @Override public Statistics visitPhysicalOlapScan(PhysicalOlapScan olapScan, Void context) { - return computeScan(olapScan); + return computeCatalogRelation(olapScan); } @Override public Statistics visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, Void context) { - return computeScan(schemaScan); + return computeCatalogRelation(schemaScan); } @Override public Statistics visitPhysicalFileScan(PhysicalFileScan fileScan, Void context) { - return computeScan(fileScan); + return computeCatalogRelation(fileScan); } @Override @@ -446,12 +446,12 @@ public Statistics visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, Void @Override public Statistics visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, Void context) { - return computeScan(jdbcScan); + return computeCatalogRelation(jdbcScan); } @Override public Statistics visitPhysicalEsScan(PhysicalEsScan esScan, Void context) { - return computeScan(esScan); + return computeCatalogRelation(esScan); } @Override @@ -586,12 +586,12 @@ private Histogram getColumnHistogram(TableIf table, String colName) { // TODO: 1. Subtract the pruned partition // 2. Consider the influence of runtime filter // 3. Get NDV and column data size from StatisticManger, StatisticManager doesn't support it now. - private Statistics computeScan(Scan scan) { - Set slotSet = scan.getOutput().stream().filter(SlotReference.class::isInstance) + private Statistics computeCatalogRelation(CatalogRelation catalogRelation) { + Set slotSet = catalogRelation.getOutput().stream().filter(SlotReference.class::isInstance) .map(s -> (SlotReference) s).collect(Collectors.toSet()); Map columnStatisticMap = new HashMap<>(); - TableIf table = scan.getTable(); - double rowCount = scan.getTable().estimatedRowCount(); + TableIf table = catalogRelation.getTable(); + double rowCount = catalogRelation.getTable().estimatedRowCount(); for (SlotReference slotReference : slotSet) { String colName = slotReference.getName(); if (colName == null) { @@ -1024,7 +1024,7 @@ public Statistics visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, Void c Preconditions.checkArgument(prodStats != null, String.format("Stats for CTE: %s not found", cteId)); Statistics consumerStats = new Statistics(prodStats.getRowCount(), new HashMap<>()); for (Slot slot : cteConsumer.getOutput()) { - Slot prodSlot = cteConsumer.findProducerSlot(slot); + Slot prodSlot = cteConsumer.getProducerSlot(slot); ColumnStatistic colStats = prodStats.columnStatistics().get(prodSlot); if (colStats == null) { continue; @@ -1060,7 +1060,7 @@ public Statistics visitPhysicalCTEConsumer(PhysicalCTEConsumer cteConsumer, Void Preconditions.checkArgument(prodStats != null, String.format("Stats for CTE: %s not found", cteId)); Statistics consumerStats = new Statistics(prodStats.getRowCount(), new HashMap<>()); for (Slot slot : cteConsumer.getOutput()) { - Slot prodSlot = cteConsumer.findProducerSlot(slot); + Slot prodSlot = cteConsumer.getProducerSlot(slot); ColumnStatistic colStats = prodStats.columnStatistics().get(prodSlot); if (colStats == null) { continue; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/DeepCopierContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/DeepCopierContext.java new file mode 100644 index 00000000000000..c4772fb7e819ed --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/DeepCopierContext.java @@ -0,0 +1,50 @@ +// 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.copier; + +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; + +import com.google.common.collect.Maps; + +import java.util.Map; + +/** + * context info used in LogicalPlan deep copy + */ +public class DeepCopierContext { + /** + * the original SlotReference to new SlotReference map + */ + public final Map exprIdReplaceMap = Maps.newHashMap(); + /** + * because LogicalApply keep original plan in itself and its right child in the meantime + * so, we must use exact same output (same ExprIds) relations between the two plan tree + * to ensure they keep same after deep copy + */ + private final Map relationReplaceMap = Maps.newHashMap(); + + public void putRelation(RelationId relationId, LogicalRelation newRelation) { + relationReplaceMap.put(relationId, newRelation); + } + + public Map getRelationReplaceMap() { + return relationReplaceMap; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/ExpressionDeepCopier.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/ExpressionDeepCopier.java new file mode 100644 index 00000000000000..0038b9e4ccad5f --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/ExpressionDeepCopier.java @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.copier; + +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Exists; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.InSubquery; +import org.apache.doris.nereids.trees.expressions.ListQuery; +import org.apache.doris.nereids.trees.expressions.ScalarSubquery; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.visitor.DefaultExpressionRewriter; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; + +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.stream.Collectors; + +/** + * deep copy expression, generate new expr id for SlotReference and Alias. + */ +public class ExpressionDeepCopier extends DefaultExpressionRewriter { + + public static ExpressionDeepCopier INSTANCE = new ExpressionDeepCopier(); + + public Expression deepCopy(Expression expression, DeepCopierContext context) { + return expression.accept(this, context); + } + + @Override + public Expression visitAlias(Alias alias, DeepCopierContext context) { + Expression child = alias.child().accept(this, context); + Map exprIdReplaceMap = context.exprIdReplaceMap; + Alias newOne; + if (exprIdReplaceMap.containsKey(alias.getExprId())) { + // NOTICE: because we do not do normalize agg, so we could get same Alias in more than one place + // so, if we already copy this Alias once, we must use the existed ExprId for this Alias. + newOne = new Alias(exprIdReplaceMap.get(alias.getExprId()), child, alias.getName()); + } else { + newOne = new Alias(child, alias.getName()); + exprIdReplaceMap.put(alias.getExprId(), newOne.getExprId()); + } + return newOne; + } + + @Override + public Expression visitSlotReference(SlotReference slotReference, DeepCopierContext context) { + Map exprIdReplaceMap = context.exprIdReplaceMap; + if (exprIdReplaceMap.containsKey(slotReference.getExprId())) { + ExprId newExprId = exprIdReplaceMap.get(slotReference.getExprId()); + return slotReference.withExprId(newExprId); + } else { + SlotReference newOne = new SlotReference(slotReference.getName(), slotReference.getDataType(), + slotReference.nullable(), slotReference.getQualifier()); + exprIdReplaceMap.put(slotReference.getExprId(), newOne.getExprId()); + return newOne; + } + } + + @Override + public Expression visitExistsSubquery(Exists exists, DeepCopierContext context) { + LogicalPlan logicalPlan = LogicalPlanDeepCopier.INSTANCE.deepCopy(exists.getQueryPlan(), context); + List correlateSlots = exists.getCorrelateSlots().stream() + .map(s -> (Slot) s.accept(this, context)) + .collect(Collectors.toList()); + Optional typeCoercionExpr = exists.getTypeCoercionExpr() + .map(c -> c.accept(this, context)); + return new Exists(logicalPlan, correlateSlots, typeCoercionExpr, exists.isNot()); + } + + @Override + public Expression visitListQuery(ListQuery listQuery, DeepCopierContext context) { + LogicalPlan logicalPlan = LogicalPlanDeepCopier.INSTANCE.deepCopy(listQuery.getQueryPlan(), context); + List correlateSlots = listQuery.getCorrelateSlots().stream() + .map(s -> (Slot) s.accept(this, context)) + .collect(Collectors.toList()); + Optional typeCoercionExpr = listQuery.getTypeCoercionExpr() + .map(c -> c.accept(this, context)); + return new ListQuery(logicalPlan, correlateSlots, typeCoercionExpr); + } + + @Override + public Expression visitInSubquery(InSubquery in, DeepCopierContext context) { + Expression compareExpr = in.getCompareExpr().accept(this, context); + List correlateSlots = in.getCorrelateSlots().stream() + .map(s -> (Slot) s.accept(this, context)) + .collect(Collectors.toList()); + Optional typeCoercionExpr = in.getTypeCoercionExpr() + .map(c -> c.accept(this, context)); + ListQuery listQuery = (ListQuery) in.getListQuery().accept(this, context); + return new InSubquery(compareExpr, listQuery, correlateSlots, typeCoercionExpr, in.isNot()); + } + + @Override + public Expression visitScalarSubquery(ScalarSubquery scalar, DeepCopierContext context) { + LogicalPlan logicalPlan = LogicalPlanDeepCopier.INSTANCE.deepCopy(scalar.getQueryPlan(), context); + List correlateSlots = scalar.getCorrelateSlots().stream() + .map(s -> (Slot) s.accept(this, context)) + .collect(Collectors.toList()); + Optional typeCoercionExpr = scalar.getTypeCoercionExpr() + .map(c -> c.accept(this, context)); + return new ScalarSubquery(logicalPlan, correlateSlots, typeCoercionExpr); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java new file mode 100644 index 00000000000000..9da4690390ee5b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java @@ -0,0 +1,429 @@ +// 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.copier; + +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.properties.OrderKey; +import org.apache.doris.nereids.trees.expressions.ExprId; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.MarkJoinSlotReference; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.expressions.SubqueryExpr; +import org.apache.doris.nereids.trees.expressions.functions.Function; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; +import org.apache.doris.nereids.trees.plans.logical.LogicalApply; +import org.apache.doris.nereids.trees.plans.logical.LogicalAssertNumRows; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; +import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; +import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalEsScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; +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.LogicalIntersect; +import org.apache.doris.nereids.trees.plans.logical.LogicalJdbcScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; +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.LogicalRepeat; +import org.apache.doris.nereids.trees.plans.logical.LogicalSchemaScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSort; +import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; +import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; +import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; +import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; + +import java.util.LinkedHashMap; +import java.util.List; +import java.util.Map; +import java.util.Optional; +import java.util.Set; + +/** + * deep copy a plan + */ +public class LogicalPlanDeepCopier extends DefaultPlanRewriter { + + public static LogicalPlanDeepCopier INSTANCE = new LogicalPlanDeepCopier(); + + public LogicalPlan deepCopy(LogicalPlan plan, DeepCopierContext context) { + return (LogicalPlan) plan.accept(this, context); + } + + @Override + public Plan visitLogicalEmptyRelation(LogicalEmptyRelation emptyRelation, DeepCopierContext context) { + List newProjects = emptyRelation.getProjects().stream() + .map(p -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(p, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalEmptyRelation(StatementScopeIdGenerator.newRelationId(), newProjects); + } + + @Override + public Plan visitLogicalOneRowRelation(LogicalOneRowRelation oneRowRelation, DeepCopierContext context) { + List newProjects = oneRowRelation.getProjects().stream() + .map(p -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(p, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalOneRowRelation(StatementScopeIdGenerator.newRelationId(), newProjects); + } + + @Override + public Plan visitLogicalApply(LogicalApply apply, DeepCopierContext context) { + Plan left = apply.left().accept(this, context); + Plan right = apply.right().accept(this, context); + List correlationSlot = apply.getCorrelationSlot().stream() + .map(s -> ExpressionDeepCopier.INSTANCE.deepCopy(s, context)) + .collect(ImmutableList.toImmutableList()); + SubqueryExpr subqueryExpr = (SubqueryExpr) ExpressionDeepCopier.INSTANCE + .deepCopy(apply.getSubqueryExpr(), context); + Optional correlationFilter = apply.getCorrelationFilter() + .map(f -> ExpressionDeepCopier.INSTANCE.deepCopy(f, context)); + Optional markJoinSlotReference = apply.getMarkJoinSlotReference() + .map(m -> (MarkJoinSlotReference) ExpressionDeepCopier.INSTANCE.deepCopy(m, context)); + Optional subCorrespondingConjunct = apply.getSubCorrespondingConjunct() + .map(c -> ExpressionDeepCopier.INSTANCE.deepCopy(c, context)); + return new LogicalApply<>(correlationSlot, subqueryExpr, correlationFilter, + markJoinSlotReference, subCorrespondingConjunct, apply.isNeedAddSubOutputToProjects(), left, right); + } + + @Override + public Plan visitLogicalAggregate(LogicalAggregate aggregate, DeepCopierContext context) { + Plan child = aggregate.child().accept(this, context); + List groupByExpressions = aggregate.getGroupByExpressions().stream() + .map(k -> ExpressionDeepCopier.INSTANCE.deepCopy(k, context)) + .collect(ImmutableList.toImmutableList()); + List outputExpressions = aggregate.getOutputExpressions().stream() + .map(o -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(o, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalAggregate<>(groupByExpressions, outputExpressions, child); + } + + @Override + public Plan visitLogicalRepeat(LogicalRepeat repeat, DeepCopierContext context) { + Plan child = repeat.child().accept(this, context); + List> groupingSets = repeat.getGroupingSets().stream() + .map(l -> l.stream() + .map(e -> ExpressionDeepCopier.INSTANCE.deepCopy(e, context)) + .collect(ImmutableList.toImmutableList())) + .collect(ImmutableList.toImmutableList()); + List outputExpressions = repeat.getOutputExpressions().stream() + .map(e -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(e, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalRepeat<>(groupingSets, outputExpressions, child); + } + + @Override + public Plan visitLogicalFilter(LogicalFilter filter, DeepCopierContext context) { + Plan child = filter.child().accept(this, context); + Set conjuncts = filter.getConjuncts().stream() + .map(p -> ExpressionDeepCopier.INSTANCE.deepCopy(p, context)) + .collect(ImmutableSet.toImmutableSet()); + return new LogicalFilter<>(conjuncts, child); + } + + @Override + public Plan visitLogicalOlapScan(LogicalOlapScan olapScan, DeepCopierContext context) { + if (context.getRelationReplaceMap().containsKey(olapScan.getRelationId())) { + return context.getRelationReplaceMap().get(olapScan.getRelationId()); + } + LogicalOlapScan newOlapScan; + if (olapScan.getManuallySpecifiedPartitions().isEmpty()) { + newOlapScan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), + olapScan.getTable(), olapScan.getQualifier(), olapScan.getHints()); + } else { + newOlapScan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), + olapScan.getTable(), olapScan.getQualifier(), + olapScan.getManuallySpecifiedPartitions(), olapScan.getHints()); + } + newOlapScan.getOutput(); + context.putRelation(olapScan.getRelationId(), newOlapScan); + updateReplaceMapWithOutput(olapScan, newOlapScan, context.exprIdReplaceMap); + return newOlapScan; + } + + @Override + public Plan visitLogicalSchemaScan(LogicalSchemaScan schemaScan, DeepCopierContext context) { + if (context.getRelationReplaceMap().containsKey(schemaScan.getRelationId())) { + return context.getRelationReplaceMap().get(schemaScan.getRelationId()); + } + LogicalSchemaScan newSchemaScan = new LogicalSchemaScan(StatementScopeIdGenerator.newRelationId(), + schemaScan.getTable(), schemaScan.getQualifier()); + updateReplaceMapWithOutput(schemaScan, newSchemaScan, context.exprIdReplaceMap); + context.putRelation(schemaScan.getRelationId(), newSchemaScan); + return newSchemaScan; + } + + @Override + public Plan visitLogicalFileScan(LogicalFileScan fileScan, DeepCopierContext context) { + if (context.getRelationReplaceMap().containsKey(fileScan.getRelationId())) { + return context.getRelationReplaceMap().get(fileScan.getRelationId()); + } + LogicalFileScan newFileScan = new LogicalFileScan(StatementScopeIdGenerator.newRelationId(), + fileScan.getTable(), fileScan.getQualifier()); + updateReplaceMapWithOutput(fileScan, newFileScan, context.exprIdReplaceMap); + context.putRelation(fileScan.getRelationId(), newFileScan); + Set conjuncts = fileScan.getConjuncts().stream() + .map(p -> ExpressionDeepCopier.INSTANCE.deepCopy(p, context)) + .collect(ImmutableSet.toImmutableSet()); + return newFileScan.withConjuncts(conjuncts); + } + + @Override + public Plan visitLogicalTVFRelation(LogicalTVFRelation tvfRelation, DeepCopierContext context) { + if (context.getRelationReplaceMap().containsKey(tvfRelation.getRelationId())) { + return context.getRelationReplaceMap().get(tvfRelation.getRelationId()); + } + LogicalTVFRelation newTVFRelation = new LogicalTVFRelation(StatementScopeIdGenerator.newRelationId(), + tvfRelation.getFunction()); + updateReplaceMapWithOutput(newTVFRelation, tvfRelation, context.exprIdReplaceMap); + context.putRelation(tvfRelation.getRelationId(), newTVFRelation); + return newTVFRelation; + } + + @Override + public Plan visitLogicalJdbcScan(LogicalJdbcScan jdbcScan, DeepCopierContext context) { + if (context.getRelationReplaceMap().containsKey(jdbcScan.getRelationId())) { + return context.getRelationReplaceMap().get(jdbcScan.getRelationId()); + } + LogicalJdbcScan newJdbcScan = new LogicalJdbcScan(StatementScopeIdGenerator.newRelationId(), + jdbcScan.getTable(), jdbcScan.getQualifier()); + updateReplaceMapWithOutput(jdbcScan, newJdbcScan, context.exprIdReplaceMap); + context.putRelation(jdbcScan.getRelationId(), newJdbcScan); + return newJdbcScan; + } + + @Override + public Plan visitLogicalEsScan(LogicalEsScan esScan, DeepCopierContext context) { + if (context.getRelationReplaceMap().containsKey(esScan.getRelationId())) { + return context.getRelationReplaceMap().get(esScan.getRelationId()); + } + LogicalEsScan newEsScan = new LogicalEsScan(StatementScopeIdGenerator.newRelationId(), + esScan.getTable(), esScan.getQualifier()); + updateReplaceMapWithOutput(esScan, newEsScan, context.exprIdReplaceMap); + context.putRelation(esScan.getRelationId(), newEsScan); + return newEsScan; + } + + @Override + public Plan visitLogicalProject(LogicalProject project, DeepCopierContext context) { + Plan child = project.child().accept(this, context); + List newProjects = project.getProjects().stream() + .map(p -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(p, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalProject<>(newProjects, child); + } + + @Override + public Plan visitLogicalSort(LogicalSort sort, DeepCopierContext context) { + Plan child = sort.child().accept(this, context); + List orderKeys = sort.getOrderKeys().stream() + .map(o -> new OrderKey(ExpressionDeepCopier.INSTANCE.deepCopy(o.getExpr(), context), + o.isAsc(), o.isNullFirst())) + .collect(ImmutableList.toImmutableList()); + return new LogicalSort<>(orderKeys, child); + } + + @Override + public Plan visitLogicalTopN(LogicalTopN topN, DeepCopierContext context) { + Plan child = topN.child().accept(this, context); + List orderKeys = topN.getOrderKeys().stream() + .map(o -> new OrderKey(ExpressionDeepCopier.INSTANCE.deepCopy(o.getExpr(), context), + o.isAsc(), o.isNullFirst())) + .collect(ImmutableList.toImmutableList()); + return new LogicalTopN<>(orderKeys, topN.getLimit(), topN.getOffset(), child); + } + + @Override + public Plan visitLogicalPartitionTopN(LogicalPartitionTopN partitionTopN, + DeepCopierContext context) { + Plan child = partitionTopN.child().accept(this, context); + List partitionKeys = partitionTopN.getPartitionKeys().stream() + .map(p -> ExpressionDeepCopier.INSTANCE.deepCopy(p, context)) + .collect(ImmutableList.toImmutableList()); + List orderKeys = partitionTopN.getOrderKeys().stream() + .map(o -> (OrderExpression) ExpressionDeepCopier.INSTANCE.deepCopy(o, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalPartitionTopN<>(partitionTopN.getFunction(), partitionKeys, orderKeys, + partitionTopN.hasGlobalLimit(), partitionTopN.getPartitionLimit(), child); + } + + @Override + public Plan visitLogicalLimit(LogicalLimit limit, DeepCopierContext context) { + Plan child = limit.child().accept(this, context); + return new LogicalLimit<>(limit.getLimit(), limit.getOffset(), limit.getPhase(), child); + } + + @Override + public Plan visitLogicalJoin(LogicalJoin join, DeepCopierContext context) { + List children = join.children().stream() + .map(c -> c.accept(this, context)) + .collect(ImmutableList.toImmutableList()); + List otherJoinConjuncts = join.getOtherJoinConjuncts().stream() + .map(c -> ExpressionDeepCopier.INSTANCE.deepCopy(c, context)) + .collect(ImmutableList.toImmutableList()); + List hashJoinConjuncts = join.getHashJoinConjuncts().stream() + .map(c -> ExpressionDeepCopier.INSTANCE.deepCopy(c, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalJoin<>(join.getJoinType(), hashJoinConjuncts, otherJoinConjuncts, + join.getHint(), join.getMarkJoinSlotReference(), children); + } + + @Override + public Plan visitLogicalAssertNumRows(LogicalAssertNumRows assertNumRows, + DeepCopierContext context) { + Plan child = assertNumRows.child().accept(this, context); + return new LogicalAssertNumRows<>(assertNumRows.getAssertNumRowsElement(), child); + } + + @Override + public Plan visitLogicalHaving(LogicalHaving having, DeepCopierContext context) { + Plan child = having.child().accept(this, context); + Set conjuncts = having.getConjuncts().stream() + .map(p -> ExpressionDeepCopier.INSTANCE.deepCopy(p, context)) + .collect(ImmutableSet.toImmutableSet()); + return new LogicalHaving<>(conjuncts, child); + } + + @Override + public Plan visitLogicalUnion(LogicalUnion union, DeepCopierContext context) { + List children = union.children().stream() + .map(c -> c.accept(this, context)) + .collect(ImmutableList.toImmutableList()); + List> constantExprsList = union.getConstantExprsList().stream() + .map(l -> l.stream() + .map(e -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(e, context)) + .collect(ImmutableList.toImmutableList())) + .collect(ImmutableList.toImmutableList()); + List outputs = union.getOutputs().stream() + .map(o -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(o, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalUnion(union.getQualifier(), outputs, constantExprsList, union.hasPushedFilter(), children); + } + + @Override + public Plan visitLogicalExcept(LogicalExcept except, DeepCopierContext context) { + List children = except.children().stream() + .map(c -> c.accept(this, context)) + .collect(ImmutableList.toImmutableList()); + List outputs = except.getOutputs().stream() + .map(o -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(o, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalExcept(except.getQualifier(), outputs, children); + } + + @Override + public Plan visitLogicalIntersect(LogicalIntersect intersect, DeepCopierContext context) { + List children = intersect.children().stream() + .map(c -> c.accept(this, context)) + .collect(ImmutableList.toImmutableList()); + List outputs = intersect.getOutputs().stream() + .map(o -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(o, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalIntersect(intersect.getQualifier(), outputs, children); + } + + @Override + public Plan visitLogicalGenerate(LogicalGenerate generate, DeepCopierContext context) { + Plan child = generate.child().accept(this, context); + List generators = generate.getGenerators().stream() + .map(g -> (Function) ExpressionDeepCopier.INSTANCE.deepCopy(g, context)) + .collect(ImmutableList.toImmutableList()); + List generatorOutput = generate.getGeneratorOutput().stream() + .map(o -> (Slot) ExpressionDeepCopier.INSTANCE.deepCopy(o, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalGenerate<>(generators, generatorOutput, child); + } + + @Override + public Plan visitLogicalWindow(LogicalWindow window, DeepCopierContext context) { + Plan child = window.child().accept(this, context); + List windowExpressions = window.getWindowExpressions().stream() + .map(w -> (NamedExpression) ExpressionDeepCopier.INSTANCE.deepCopy(w, context)) + .collect(ImmutableList.toImmutableList()); + return new LogicalWindow<>(windowExpressions, child); + } + + @Override + public Plan visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, + DeepCopierContext context) { + Plan child = olapTableSink.child().accept(this, context); + return new LogicalOlapTableSink<>(olapTableSink.getDatabase(), olapTableSink.getTargetTable(), + olapTableSink.getCols(), olapTableSink.getPartitionIds(), child); + } + + @Override + public Plan visitLogicalFileSink(LogicalFileSink fileSink, DeepCopierContext context) { + Plan child = fileSink.child().accept(this, context); + return fileSink.withChildren(child); + } + + @Override + public Plan visitLogicalCTEProducer(LogicalCTEProducer cteProducer, DeepCopierContext context) { + throw new AnalysisException("plan deep copier could not copy CTEProducer."); + } + + @Override + public Plan visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, DeepCopierContext context) { + if (context.getRelationReplaceMap().containsKey(cteConsumer.getRelationId())) { + return context.getRelationReplaceMap().get(cteConsumer.getRelationId()); + } + Map consumerToProducerOutputMap = new LinkedHashMap<>(); + Map producerToConsumerOutputMap = new LinkedHashMap<>(); + for (Slot consumerOutput : cteConsumer.getOutput()) { + Slot newOutput = (Slot) ExpressionDeepCopier.INSTANCE.deepCopy(consumerOutput, context); + consumerToProducerOutputMap.put(newOutput, cteConsumer.getProducerSlot(consumerOutput)); + producerToConsumerOutputMap.put(cteConsumer.getProducerSlot(consumerOutput), newOutput); + } + LogicalCTEConsumer newCTEConsumer = new LogicalCTEConsumer( + StatementScopeIdGenerator.newRelationId(), + cteConsumer.getCteId(), cteConsumer.getName(), + consumerToProducerOutputMap, producerToConsumerOutputMap); + context.putRelation(cteConsumer.getRelationId(), newCTEConsumer); + return newCTEConsumer; + } + + @Override + public Plan visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, + DeepCopierContext context) { + throw new AnalysisException("plan deep copier could not copy CTEAnchor."); + } + + private void updateReplaceMapWithOutput(Plan oldPlan, Plan newPlan, Map replaceMap) { + List oldOutput = oldPlan.getOutput(); + List newOutput = newPlan.getOutput(); + for (int i = 0; i < newOutput.size(); i++) { + replaceMap.put(oldOutput.get(i).getExprId(), newOutput.get(i).getExprId()); + } + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Exists.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Exists.java index 28762addd7984d..5fba8c5687e444 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Exists.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Exists.java @@ -34,11 +34,12 @@ * Exists subquery expression. */ public class Exists extends SubqueryExpr implements LeafExpression { + private final boolean isNot; public Exists(LogicalPlan subquery, boolean isNot) { super(Objects.requireNonNull(subquery, "subquery can not be null")); - this.isNot = Objects.requireNonNull(isNot, "isNot can not be null"); + this.isNot = isNot; } public Exists(LogicalPlan subquery, List correlateSlots, boolean isNot) { @@ -52,7 +53,7 @@ public Exists(LogicalPlan subquery, List correlateSlots, super(Objects.requireNonNull(subquery, "subquery can not be null"), Objects.requireNonNull(correlateSlots, "subquery can not be null"), typeCoercionExpr); - this.isNot = Objects.requireNonNull(isNot, "isNot can not be null"); + this.isNot = isNot; } public boolean isNot() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MarkJoinSlotReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MarkJoinSlotReference.java index ff841bdeb2558c..099e64eb5d2ff4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MarkJoinSlotReference.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/MarkJoinSlotReference.java @@ -20,6 +20,8 @@ import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor; import org.apache.doris.nereids.types.BooleanType; +import com.google.common.collect.ImmutableList; + /** * A special type of column that will be generated to replace the subquery when unnesting the subquery of MarkJoin. */ @@ -36,6 +38,11 @@ public MarkJoinSlotReference(String name, boolean existsHasAgg) { this.existsHasAgg = existsHasAgg; } + public MarkJoinSlotReference(ExprId exprId, String name, boolean existsHasAgg) { + super(exprId, name, BooleanType.INSTANCE, false, ImmutableList.of()); + this.existsHasAgg = existsHasAgg; + } + @Override public R accept(ExpressionVisitor visitor, C context) { return visitor.visitMarkJoinReference(this, context); @@ -61,4 +68,9 @@ public boolean equals(Object o) { public boolean isExistsHasAgg() { return existsHasAgg; } + + @Override + public MarkJoinSlotReference withExprId(ExprId exprId) { + return new MarkJoinSlotReference(exprId, name, existsHasAgg); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Slot.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Slot.java index 3a3b498c519f80..c5dc57018fc857 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Slot.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Slot.java @@ -35,11 +35,15 @@ public Slot withNullable(boolean newNullable) { throw new RuntimeException("Do not implement"); } - public Slot withQualifier(List qualifiers) { + public Slot withQualifier(List qualifier) { throw new RuntimeException("Do not implement"); } public Slot withName(String name) { throw new RuntimeException("Do not implement"); } + + public Slot withExprId(ExprId exprId) { + throw new RuntimeException("Do not implement"); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java index 1c1452b39927f3..b76cc77a51c8da 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SlotReference.java @@ -33,13 +33,11 @@ * Reference to slot in expression. */ public class SlotReference extends Slot { - private final ExprId exprId; - // TODO: we should distinguish the name is alias or column name, and the column name should contains - // `cluster:db`.`table`.`column` - private final String name; - private final DataType dataType; - private final boolean nullable; - private final List qualifier; + protected final ExprId exprId; + protected final String name; + protected final DataType dataType; + protected final boolean nullable; + protected final List qualifier; private final Column column; public SlotReference(String name, DataType dataType) { @@ -182,6 +180,7 @@ public SlotReference withChildren(List children) { return this; } + @Override public SlotReference withNullable(boolean newNullable) { if (this.nullable == newNullable) { return this; @@ -190,16 +189,21 @@ public SlotReference withNullable(boolean newNullable) { } @Override - public SlotReference withQualifier(List qualifiers) { - return new SlotReference(exprId, name, dataType, nullable, qualifiers, column); + public SlotReference withQualifier(List qualifier) { + return new SlotReference(exprId, name, dataType, nullable, qualifier, column); } - public boolean isVisible() { - return column == null || column.isVisible(); + @Override + public SlotReference withName(String name) { + return new SlotReference(exprId, name, dataType, nullable, qualifier, column); } @Override - public Slot withName(String name) { + public SlotReference withExprId(ExprId exprId) { return new SlotReference(exprId, name, dataType, nullable, qualifier, column); } + + public boolean isVisible() { + return column == null || column.isVisible(); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StatementScopeIdGenerator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StatementScopeIdGenerator.java index ed9fceeb1123d1..f5aed7e58228ee 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StatementScopeIdGenerator.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/StatementScopeIdGenerator.java @@ -19,6 +19,7 @@ import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.qe.ConnectContext; import com.google.common.annotations.VisibleForTesting; @@ -47,6 +48,14 @@ public static ObjectId newObjectId() { return ConnectContext.get().getStatementContext().getNextObjectId(); } + public static RelationId newRelationId() { + // this branch is for test only + if (ConnectContext.get() == null || ConnectContext.get().getStatementContext() == null) { + return statementContext.getNextRelationId(); + } + return ConnectContext.get().getStatementContext().getNextRelationId(); + } + public static CTEId newCTEId() { // this branch is for test only if (ConnectContext.get() == null || ConnectContext.get().getStatementContext() == null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SubqueryExpr.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SubqueryExpr.java index b783918209ef93..44ea635d561c12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SubqueryExpr.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/SubqueryExpr.java @@ -62,6 +62,10 @@ public Expression getSubqueryOutput() { return typeCoercionExpr.orElseGet(() -> queryPlan.getOutput().get(0)); } + public Expression getSubqueryOutput(LogicalPlan queryPlan) { + return typeCoercionExpr.orElseGet(() -> queryPlan.getOutput().get(0)); + } + @Override public DataType getDataType() throws UnboundException { throw new UnboundException("getDataType"); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/VirtualSlotReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/VirtualSlotReference.java index 76906d21390cce..eb0829bf0e4599 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/VirtualSlotReference.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/VirtualSlotReference.java @@ -119,4 +119,30 @@ public int hashCode() { public boolean nullable() { return false; } + + public VirtualSlotReference withNullable(boolean newNullable) { + if (this.nullable == newNullable) { + return this; + } + return new VirtualSlotReference(exprId, name, dataType, newNullable, qualifier, + originExpression, computeLongValueMethod); + } + + @Override + public VirtualSlotReference withQualifier(List qualifier) { + return new VirtualSlotReference(exprId, name, dataType, nullable, qualifier, + originExpression, computeLongValueMethod); + } + + @Override + public VirtualSlotReference withName(String name) { + return new VirtualSlotReference(exprId, name, dataType, nullable, qualifier, + originExpression, computeLongValueMethod); + } + + @Override + public VirtualSlotReference withExprId(ExprId exprId) { + return new VirtualSlotReference(exprId, name, dataType, nullable, qualifier, + originExpression, computeLongValueMethod); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java index ff203c9b2f9f64..eae1e25201a144 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/table/TableValuedFunction.java @@ -41,7 +41,8 @@ /** TableValuedFunction */ public abstract class TableValuedFunction extends BoundFunction implements UnaryExpression, CustomSignature { - protected final Supplier catalogFunctionCache = Suppliers.memoize(() -> toCatalogFunction()); + + protected final Supplier catalogFunctionCache = Suppliers.memoize(this::toCatalogFunction); protected final Supplier tableCache = Suppliers.memoize(() -> { try { return catalogFunctionCache.get().getTable(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/ObjectId.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/ObjectId.java index a540afd0ce52be..c1f58361b3198e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/ObjectId.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/ObjectId.java @@ -63,6 +63,6 @@ public int hashCode() { @Override public String toString() { - return "RelationId#" + id; + return "ObjectId#" + id; } } 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 05bebf74033481..abefbce49292df 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 @@ -75,7 +75,7 @@ default LogicalProperties computeLogicalProperties() { /** * Get extra plans. */ - default List extraPlans() { + default List extraPlans() { return ImmutableList.of(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/RelationId.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/RelationId.java new file mode 100644 index 00000000000000..45e2ab7ee095f3 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/RelationId.java @@ -0,0 +1,68 @@ +// 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; + +import org.apache.doris.common.Id; +import org.apache.doris.common.IdGenerator; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; + +import java.util.Objects; + +/** + * relation id + */ +public class RelationId extends Id { + + public RelationId(int id) { + super(id); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + RelationId relationId = (RelationId) o; + return id == relationId.id; + } + + /** + * Should be only called by {@link StatementScopeIdGenerator}. + */ + public static IdGenerator createGenerator() { + return new IdGenerator() { + @Override + public RelationId getNextId() { + return new RelationId(nextId++); + } + }; + } + + @Override + public int hashCode() { + return Objects.hash(id); + } + + @Override + public String toString() { + return "RelationId#" + id; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/CatalogRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/CatalogRelation.java index cef0b3fce47106..f009c413344c32 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/CatalogRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/CatalogRelation.java @@ -18,12 +18,13 @@ package org.apache.doris.nereids.trees.plans.algebra; import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.TableIf; import org.apache.doris.nereids.exceptions.AnalysisException; /** CatalogRelation */ public interface CatalogRelation extends Relation { - Table getTable(); + + TableIf getTable(); Database getDatabase() throws AnalysisException; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/OlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/OlapScan.java index d0074c4119e08f..d5fe7c2341369d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/OlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/OlapScan.java @@ -22,7 +22,8 @@ import java.util.List; /** OlapScan */ -public interface OlapScan extends Scan { +public interface OlapScan { + OlapTable getTable(); long getSelectedIndexId(); @@ -39,10 +40,10 @@ default int getScanTabletNum() { } OlapTable olapTable = getTable(); - Integer selectTabletNumInPartitions = getSelectedPartitionIds().stream() - .map(partitionId -> olapTable.getPartition(partitionId)) + int selectTabletNumInPartitions = getSelectedPartitionIds().stream() + .map(olapTable::getPartition) .map(partition -> partition.getDistributionInfo().getBucketNum()) - .reduce((b1, b2) -> b1 + b2) + .reduce(Integer::sum) .orElse(0); if (selectTabletNumInPartitions > 0) { return selectTabletNumInPartitions; @@ -52,7 +53,7 @@ default int getScanTabletNum() { return olapTable.getAllPartitions() .stream() .map(partition -> partition.getDistributionInfo().getBucketNum()) - .reduce((b1, b2) -> b1 + b2) + .reduce(Integer::sum) .orElse(0); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Relation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Relation.java index f0f0ab844ec2f5..1c95e33ff63f4a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Relation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Relation.java @@ -18,6 +18,7 @@ package org.apache.doris.nereids.trees.plans.algebra; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.RelationId; import java.util.List; @@ -25,5 +26,8 @@ * Relation base interface */ public interface Relation { + + RelationId getRelationId(); + List getOutput(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Scan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Scan.java deleted file mode 100644 index 5f0baf2232b09c..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Scan.java +++ /dev/null @@ -1,27 +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.trees.plans.algebra; - -import org.apache.doris.catalog.TableIf; - -/** - * Common interface for logical/physical scan. - */ -public interface Scan extends Relation { - TableIf getTable(); -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java index 89ef68b5617f1f..7504863ea9589e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java @@ -34,7 +34,7 @@ */ public abstract class AbstractLogicalPlan extends AbstractPlan implements LogicalPlan, Explainable { - private Supplier hasUnboundExpressions = () -> super.hasUnboundExpression(); + private final Supplier hasUnboundExpressions = super::hasUnboundExpression; public AbstractLogicalPlan(PlanType type, Plan... children) { super(type, children); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java index b40e2e2c5b42ae..6672f9e81aaf5e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java @@ -61,7 +61,7 @@ public class LogicalAggregate private final List outputExpressions; // When there are grouping sets/rollup/cube, LogicalAgg is generated by LogicalRepeat. - private final Optional sourceRepeat; + private final Optional> sourceRepeat; private final boolean ordinalIsResolved; @@ -89,7 +89,7 @@ public LogicalAggregate(List groupByExpressions, public LogicalAggregate( List groupByExpressions, List outputExpressions, - Optional sourceRepeat, + Optional> sourceRepeat, CHILD_TYPE child) { this(groupByExpressions, outputExpressions, false, sourceRepeat, child); } @@ -98,7 +98,7 @@ public LogicalAggregate( List groupByExpressions, List outputExpressions, boolean normalized, - Optional sourceRepeat, + Optional> sourceRepeat, CHILD_TYPE child) { this(groupByExpressions, outputExpressions, normalized, false, sourceRepeat, Optional.empty(), Optional.empty(), child); @@ -136,7 +136,7 @@ public String getOutputExprsSql() { return outputExpressions.stream().map(ExpressionTrait::toSql).collect(Collectors.joining(", ")); } - public Optional getSourceRepeat() { + public Optional> getSourceRepeat() { return sourceRepeat; } @@ -191,7 +191,7 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - LogicalAggregate that = (LogicalAggregate) o; + LogicalAggregate that = (LogicalAggregate) o; return Objects.equals(groupByExpressions, that.groupByExpressions) && Objects.equals(outputExpressions, that.outputExpressions) && normalized == that.normalized diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTE.java index 9b2727faabecb5..e21726ad7c92f9 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTE.java @@ -19,7 +19,6 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; -import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; @@ -29,11 +28,8 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; -import com.google.common.collect.ImmutableMap; -import java.util.HashMap; import java.util.List; -import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -44,35 +40,14 @@ public class LogicalCTE extends LogicalUnary> aliasQueries; - private final Map cteNameToId; - - private final boolean registered; - public LogicalCTE(List> aliasQueries, CHILD_TYPE child) { - this(aliasQueries, Optional.empty(), Optional.empty(), child, false, null); - } - - public LogicalCTE(List> aliasQueries, CHILD_TYPE child, boolean registered, - Map cteNameToId) { - this(aliasQueries, Optional.empty(), Optional.empty(), child, registered, - cteNameToId); + this(aliasQueries, Optional.empty(), Optional.empty(), child); } public LogicalCTE(List> aliasQueries, Optional groupExpression, - Optional logicalProperties, CHILD_TYPE child, - boolean registered, Map cteNameToId) { + Optional logicalProperties, CHILD_TYPE child) { super(PlanType.LOGICAL_CTE, groupExpression, logicalProperties, child); this.aliasQueries = ImmutableList.copyOf(Objects.requireNonNull(aliasQueries, "aliasQueries can not be null")); - this.registered = registered; - this.cteNameToId = cteNameToId == null ? ImmutableMap.copyOf(initCTEId()) : cteNameToId; - } - - private Map initCTEId() { - Map subQueryAliasToUniqueId = new HashMap<>(); - for (LogicalSubQueryAlias subQueryAlias : aliasQueries) { - subQueryAliasToUniqueId.put(subQueryAlias.getAlias(), subQueryAlias.getCteId()); - } - return subQueryAliasToUniqueId; } public List> getAliasQueries() { @@ -80,8 +55,8 @@ public List> getAliasQueries() { } @Override - public List extraPlans() { - return (List) aliasQueries; + public List extraPlans() { + return aliasQueries; } /** @@ -126,7 +101,7 @@ public int hashCode() { @Override public Plan withChildren(List children) { Preconditions.checkArgument(aliasQueries.size() > 0); - return new LogicalCTE<>(aliasQueries, children.get(0), registered, cteNameToId); + return new LogicalCTE<>(aliasQueries, children.get(0)); } @Override @@ -141,30 +116,13 @@ public List getExpressions() { @Override public LogicalCTE withGroupExpression(Optional groupExpression) { - return new LogicalCTE<>(aliasQueries, groupExpression, Optional.of(getLogicalProperties()), child(), - registered, cteNameToId); + return new LogicalCTE<>(aliasQueries, groupExpression, Optional.of(getLogicalProperties()), child()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { Preconditions.checkArgument(aliasQueries.size() > 0); - return new LogicalCTE<>(aliasQueries, groupExpression, logicalProperties, children.get(0), - registered, cteNameToId); - } - - public boolean isRegistered() { - return registered; - } - - public CTEId findCTEId(String subQueryAlias) { - CTEId id = cteNameToId.get(subQueryAlias); - Preconditions.checkArgument(id != null, "Cannot find id for sub-query : %s", - subQueryAlias); - return id; - } - - public Map getCteNameToId() { - return cteNameToId; + return new LogicalCTE<>(aliasQueries, groupExpression, logicalProperties, children.get(0)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEAnchor.java index 605554efc109d6..c7b6bb5158ad92 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEAnchor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEAnchor.java @@ -40,21 +40,19 @@ public class LogicalCTEAnchor groupExpression, - Optional logicalProperties, - LEFT_CHILD_TYPE leftChild, RIGHT_CHILD_TYPE rightChild, CTEId cteId) { + public LogicalCTEAnchor(CTEId cteId, Optional groupExpression, + Optional logicalProperties, LEFT_CHILD_TYPE leftChild, RIGHT_CHILD_TYPE rightChild) { super(PlanType.LOGICAL_CTE_ANCHOR, groupExpression, logicalProperties, leftChild, rightChild); this.cteId = cteId; } @Override public Plan withChildren(List children) { - return new LogicalCTEAnchor<>(groupExpression, Optional.of(getLogicalProperties()), - children.get(0), children.get(1), cteId); + return new LogicalCTEAnchor<>(cteId, children.get(0), children.get(1)); } @Override @@ -69,13 +67,13 @@ public List getExpressions() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new LogicalCTEAnchor<>(groupExpression, Optional.of(getLogicalProperties()), left(), right(), cteId); + return new LogicalCTEAnchor<>(cteId, groupExpression, Optional.of(getLogicalProperties()), left(), right()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalCTEAnchor<>(groupExpression, logicalProperties, children.get(0), children.get(1), cteId); + return new LogicalCTEAnchor<>(cteId, groupExpression, logicalProperties, children.get(0), children.get(1)); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java index 33e688645db076..86f58ce4c1d271 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java @@ -20,12 +20,11 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.trees.expressions.CTEId; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -35,59 +34,67 @@ import java.util.LinkedHashMap; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Optional; /** * LogicalCTEConsumer */ -public class LogicalCTEConsumer extends LogicalLeaf { - - private final CTEId cteId; - - private final Map consumerToProducerOutputMap = new LinkedHashMap<>(); - - private final Map producerToConsumerOutputMap = new LinkedHashMap<>(); - - private final int consumerId; +public class LogicalCTEConsumer extends LogicalRelation { private final String name; + private final CTEId cteId; + private final Map consumerToProducerOutputMap; + private final Map producerToConsumerOutputMap; /** * Logical CTE consumer. */ - public LogicalCTEConsumer(Optional groupExpression, - Optional logicalProperties, LogicalPlan childPlan, CTEId cteId, String name) { - super(PlanType.LOGICAL_CTE_RELATION, groupExpression, logicalProperties); - this.cteId = cteId; - this.name = name; - initProducerToConsumerOutputMap(childPlan); - for (Map.Entry entry : producerToConsumerOutputMap.entrySet()) { - this.consumerToProducerOutputMap.put(entry.getValue(), entry.getKey()); - } - this.consumerId = StatementScopeIdGenerator.newCTEId().asInt(); + public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, + Map consumerToProducerOutputMap, Map producerToConsumerOutputMap) { + super(relationId, PlanType.LOGICAL_CTE_RELATION, Optional.empty(), Optional.empty()); + this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); + this.name = Objects.requireNonNull(name, "name should not null"); + this.consumerToProducerOutputMap = Objects.requireNonNull(consumerToProducerOutputMap, + "consumerToProducerOutputMap should not null"); + this.producerToConsumerOutputMap = Objects.requireNonNull(producerToConsumerOutputMap, + "producerToConsumerOutputMap should not null"); } /** * Logical CTE consumer. */ - public LogicalCTEConsumer(Optional groupExpression, - Optional logicalProperties, CTEId cteId, - Map consumerToProducerOutputMap, - Map producerToConsumerOutputMap, int consumerId, String name) { - super(PlanType.LOGICAL_CTE_RELATION, groupExpression, logicalProperties); - this.cteId = cteId; - this.consumerToProducerOutputMap.putAll(consumerToProducerOutputMap); - this.producerToConsumerOutputMap.putAll(producerToConsumerOutputMap); - this.consumerId = consumerId; - this.name = name; + public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, LogicalPlan producerPlan) { + super(relationId, PlanType.LOGICAL_CTE_RELATION, Optional.empty(), Optional.empty()); + this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); + this.name = Objects.requireNonNull(name, "name should not null"); + this.consumerToProducerOutputMap = new LinkedHashMap<>(); + this.producerToConsumerOutputMap = new LinkedHashMap<>(); + initOutputMaps(producerPlan); } - private void initProducerToConsumerOutputMap(LogicalPlan childPlan) { + /** + * Logical CTE consumer. + */ + public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, + Map consumerToProducerOutputMap, Map producerToConsumerOutputMap, + Optional groupExpression, Optional logicalProperties) { + super(relationId, PlanType.LOGICAL_CTE_RELATION, groupExpression, logicalProperties); + this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); + this.name = Objects.requireNonNull(name, "name should not null"); + this.consumerToProducerOutputMap = Objects.requireNonNull(consumerToProducerOutputMap, + "consumerToProducerOutputMap should not null"); + this.producerToConsumerOutputMap = Objects.requireNonNull(producerToConsumerOutputMap, + "producerToConsumerOutputMap should not null"); + } + + private void initOutputMaps(LogicalPlan childPlan) { List producerOutput = childPlan.getOutput(); for (Slot producerOutputSlot : producerOutput) { Slot consumerSlot = new SlotReference(producerOutputSlot.getName(), producerOutputSlot.getDataType(), producerOutputSlot.nullable(), ImmutableList.of(name)); producerToConsumerOutputMap.put(producerOutputSlot, consumerSlot); + consumerToProducerOutputMap.put(consumerSlot, producerOutputSlot); } } @@ -104,26 +111,25 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitLogicalCTEConsumer(this, context); } - @Override - public List getExpressions() { - return ImmutableList.of(); + public Plan withTwoMaps(Map consumerToProducerOutputMap, Map producerToConsumerOutputMap) { + return new LogicalCTEConsumer(relationId, cteId, name, + consumerToProducerOutputMap, producerToConsumerOutputMap, + Optional.empty(), Optional.empty()); } @Override public Plan withGroupExpression(Optional groupExpression) { - return new LogicalCTEConsumer(groupExpression, Optional.of(getLogicalProperties()), cteId, - consumerToProducerOutputMap, - producerToConsumerOutputMap, - consumerId, name); + return new LogicalCTEConsumer(relationId, cteId, name, + consumerToProducerOutputMap, producerToConsumerOutputMap, + groupExpression, Optional.of(getLogicalProperties())); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalCTEConsumer(groupExpression, logicalProperties, cteId, - consumerToProducerOutputMap, - producerToConsumerOutputMap, - consumerId, name); + return new LogicalCTEConsumer(relationId, cteId, name, + consumerToProducerOutputMap, producerToConsumerOutputMap, + groupExpression, logicalProperties); } @Override @@ -135,31 +141,11 @@ public CTEId getCteId() { return cteId; } - @Override - public int hashCode() { - return consumerId; - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - return this.consumerId == ((LogicalCTEConsumer) o).consumerId; - } - - public int getConsumerId() { - return consumerId; - } - public String getName() { return name; } - public Slot findProducerSlot(Slot consumerSlot) { + public Slot getProducerSlot(Slot consumerSlot) { Slot slot = consumerToProducerOutputMap.get(consumerSlot); Preconditions.checkArgument(slot != null, String.format("Required producer" + "slot for :%s doesn't exist", consumerSlot)); @@ -170,6 +156,7 @@ public Slot findProducerSlot(Slot consumerSlot) { public String toString() { return Utils.toSqlString("LogicalCteConsumer[" + id.asInt() + "]", "cteId", cteId, - "consumerId", consumerId); + "relationId", relationId, + "name", name); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEProducer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEProducer.java index b6123edacaf54a..089f1ec9fa9aba 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEProducer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEProducer.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -40,45 +41,30 @@ public class LogicalCTEProducer extends LogicalUnary projects; - - private final boolean rewritten; - - public LogicalCTEProducer(CHILD_TYPE child, CTEId cteId) { + public LogicalCTEProducer(CTEId cteId, CHILD_TYPE child) { super(PlanType.LOGICAL_CTE_PRODUCER, child); this.cteId = cteId; - this.projects = ImmutableList.of(); - this.rewritten = false; } - public LogicalCTEProducer(Optional groupExpression, - Optional logicalProperties, CHILD_TYPE child, CTEId cteId, - List projects, boolean rewritten) { + public LogicalCTEProducer(CTEId cteId, Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { super(PlanType.LOGICAL_CTE_PRODUCER, groupExpression, logicalProperties, child); this.cteId = cteId; - this.projects = ImmutableList.copyOf(Objects.requireNonNull(projects, - "projects should not null")); - this.rewritten = rewritten; } public CTEId getCteId() { return cteId; } - public List getProjects() { - return projects; - } - @Override public Plan withChildren(List children) { Preconditions.checkArgument(children.size() == 1); - return new LogicalCTEProducer<>(groupExpression, Optional.of(getLogicalProperties()), children.get(0), - cteId, projects, rewritten); + return new LogicalCTEProducer<>(cteId, children.get(0)); } - public Plan withChildrenAndProjects(List children, List projects, boolean rewritten) { - return new LogicalCTEProducer<>(groupExpression, Optional.of(getLogicalProperties()), children.get(0), - cteId, projects, rewritten); + @Override + public List getExpressions() { + return ImmutableList.of(); } @Override @@ -86,41 +72,26 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitLogicalCTEProducer(this, context); } - @Override - public List getExpressions() { - return child().getExpressions(); - } - @Override public Plan withGroupExpression(Optional groupExpression) { - return new LogicalCTEProducer<>(groupExpression, Optional.of(getLogicalProperties()), child(), cteId, - projects, rewritten); + return new LogicalCTEProducer<>(cteId, groupExpression, Optional.of(getLogicalProperties()), child()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalCTEProducer<>(groupExpression, logicalProperties, children.get(0), cteId, - projects, rewritten); + return new LogicalCTEProducer<>(cteId, groupExpression, logicalProperties, children.get(0)); } @Override public List computeOutput() { - return child().computeOutput(); + return child().getOutput(); } @Override public String toString() { - return String.format("LOGICAL_CTE_PRODUCER#%d", cteId.asInt()); - } - - public boolean isRewritten() { - return rewritten; - } - - @Override - public int hashCode() { - return Objects.hash(cteId, projects, rewritten); + return Utils.toSqlString("LogicalCteProducer[" + id.asInt() + "]", + "cteId", cteId); } @Override @@ -131,13 +102,15 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - LogicalCTEProducer p = (LogicalCTEProducer) o; - if (cteId != p.cteId) { - return false; - } - if (rewritten != p.rewritten) { + if (!super.equals(o)) { return false; } - return projects.equals(p.projects); + LogicalCTEProducer that = (LogicalCTEProducer) o; + return Objects.equals(cteId, that.cteId); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), cteId); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCatalogRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCatalogRelation.java new file mode 100644 index 00000000000000..6150e9047aca60 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCatalogRelation.java @@ -0,0 +1,98 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.logical; + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * abstract class catalog relation for logical relation + */ +public abstract class LogicalCatalogRelation extends LogicalRelation implements CatalogRelation { + + protected final TableIf table; + protected final ImmutableList qualifier; + + public LogicalCatalogRelation(RelationId relationId, PlanType type, TableIf table, List qualifier) { + super(relationId, type); + this.table = Objects.requireNonNull(table, "table can not be null"); + this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier, "qualifier can not be null")); + } + + public LogicalCatalogRelation(RelationId relationId, PlanType type, TableIf table, List qualifier, + Optional groupExpression, Optional logicalProperties) { + super(relationId, type, groupExpression, logicalProperties); + this.table = Objects.requireNonNull(table, "table can not be null"); + this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier, "qualifier can not be null")); + } + + @Override + public TableIf getTable() { + return table; + } + + @Override + public Database getDatabase() throws AnalysisException { + Preconditions.checkArgument(!qualifier.isEmpty()); + return Env.getCurrentInternalCatalog().getDbOrException(qualifier.get(0), + s -> new AnalysisException("Database [" + qualifier.get(0) + "] does not exist.")); + } + + @Override + public List computeOutput() { + return table.getBaseSchema() + .stream() + .map(col -> SlotReference.fromColumn(col, qualified())) + .collect(ImmutableList.toImmutableList()); + } + + public List getQualifier() { + return qualifier; + } + + /** + * Full qualified name parts, i.e., concat qualifier and name into a list. + */ + public List qualified() { + return Utils.qualifiedNameParts(qualifier, table.getName()); + } + + /** + * Full qualified table name, concat qualifier and name with `.` as separator. + */ + public String qualifiedName() { + return Utils.qualifiedName(qualifier, table.getName()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalEmptyRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalEmptyRelation.java index 3484b959836070..32e7e1144ac09c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalEmptyRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalEmptyRelation.java @@ -19,11 +19,11 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.EmptyRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -39,17 +39,17 @@ * e.g. * select * from tbl limit 0 */ -public class LogicalEmptyRelation extends LogicalLeaf implements EmptyRelation, OutputPrunable { +public class LogicalEmptyRelation extends LogicalRelation implements EmptyRelation, OutputPrunable { private final List projects; - public LogicalEmptyRelation(List projects) { - this(projects, Optional.empty(), Optional.empty()); + public LogicalEmptyRelation(RelationId relationId, List projects) { + this(relationId, projects, Optional.empty(), Optional.empty()); } - public LogicalEmptyRelation(List projects, Optional groupExpression, - Optional logicalProperties) { - super(PlanType.LOGICAL_ONE_ROW_RELATION, groupExpression, logicalProperties); + public LogicalEmptyRelation(RelationId relationId, List projects, + Optional groupExpression, Optional logicalProperties) { + super(relationId, PlanType.LOGICAL_ONE_ROW_RELATION, groupExpression, logicalProperties); this.projects = ImmutableList.copyOf(Objects.requireNonNull(projects, "projects can not be null")); } @@ -63,24 +63,20 @@ public List getProjects() { return projects; } - @Override - public List getExpressions() { - return ImmutableList.of(); - } - public LogicalEmptyRelation withProjects(List projects) { - return new LogicalEmptyRelation(projects, Optional.empty(), Optional.empty()); + return new LogicalEmptyRelation(relationId, projects, Optional.empty(), Optional.empty()); } @Override public Plan withGroupExpression(Optional groupExpression) { - return new LogicalEmptyRelation(projects, groupExpression, Optional.of(logicalPropertiesSupplier.get())); + return new LogicalEmptyRelation(relationId, projects, + groupExpression, Optional.of(logicalPropertiesSupplier.get())); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalEmptyRelation(projects, groupExpression, logicalProperties); + return new LogicalEmptyRelation(relationId, projects, groupExpression, logicalProperties); } @Override @@ -97,26 +93,6 @@ public String toString() { ); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - LogicalEmptyRelation that = (LogicalEmptyRelation) o; - return Objects.equals(projects, that.projects); - } - - @Override - public int hashCode() { - return Objects.hash(projects); - } - @Override public List getOutputs() { return projects; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalEsScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalEsScan.java index 012348938bff31..d34f47266f7337 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalEsScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalEsScan.java @@ -20,9 +20,9 @@ import org.apache.doris.catalog.external.ExternalTable; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -34,19 +34,18 @@ /** * Logical scan for external es catalog. */ -public class LogicalEsScan extends LogicalRelation { +public class LogicalEsScan extends LogicalCatalogRelation { /** * Constructor for LogicalEsScan. */ - public LogicalEsScan(ObjectId id, ExternalTable table, List qualifier, + public LogicalEsScan(RelationId id, ExternalTable table, List qualifier, Optional groupExpression, Optional logicalProperties) { - super(id, PlanType.LOGICAL_ES_SCAN, table, qualifier, - groupExpression, logicalProperties); + super(id, PlanType.LOGICAL_ES_SCAN, table, qualifier, groupExpression, logicalProperties); } - public LogicalEsScan(ObjectId id, ExternalTable table, List qualifier) { + public LogicalEsScan(RelationId id, ExternalTable table, List qualifier) { this(id, table, qualifier, Optional.empty(), Optional.empty()); } @@ -66,14 +65,14 @@ public String toString() { @Override public LogicalEsScan withGroupExpression(Optional groupExpression) { - return new LogicalEsScan(id, (ExternalTable) table, qualifier, groupExpression, + return new LogicalEsScan(relationId, (ExternalTable) table, qualifier, groupExpression, Optional.of(getLogicalProperties())); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalEsScan(id, (ExternalTable) table, qualifier, groupExpression, logicalProperties); + return new LogicalEsScan(relationId, (ExternalTable) table, qualifier, groupExpression, logicalProperties); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java index 52d66de129d14e..af5dcaa0f0284b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileScan.java @@ -21,9 +21,9 @@ 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.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -31,67 +31,63 @@ import com.google.common.collect.Sets; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.Set; /** * Logical file scan for external catalog. */ -public class LogicalFileScan extends LogicalRelation { +public class LogicalFileScan extends LogicalCatalogRelation { + // TODO remove this conjuncts. private final Set conjuncts; /** * Constructor for LogicalFileScan. */ - public LogicalFileScan(ObjectId id, ExternalTable table, List qualifier, - Optional groupExpression, - Optional logicalProperties, - Set conjuncts) { + public LogicalFileScan(RelationId id, ExternalTable table, List qualifier, + Optional groupExpression, Optional logicalProperties, + Set conjuncts) { super(id, PlanType.LOGICAL_FILE_SCAN, table, qualifier, groupExpression, logicalProperties); this.conjuncts = conjuncts; } - public LogicalFileScan(ObjectId id, ExternalTable table, List qualifier) { + public LogicalFileScan(RelationId id, ExternalTable table, List qualifier) { this(id, table, qualifier, Optional.empty(), Optional.empty(), Sets.newHashSet()); } @Override public ExternalTable getTable() { - Preconditions.checkArgument(table instanceof ExternalTable); + Preconditions.checkArgument(table instanceof ExternalTable, + "LogicalFileScan's table must be ExternalTable, but table is " + table.getClass().getSimpleName()); return (ExternalTable) table; } @Override public String toString() { return Utils.toSqlString("LogicalFileScan", - "qualified", qualifiedName(), - "output", getOutput() + "qualified", qualifiedName(), + "output", getOutput() ); } - @Override - public boolean equals(Object o) { - return super.equals(o) && Objects.equals(conjuncts, ((LogicalFileScan) o).conjuncts); - } - @Override public LogicalFileScan withGroupExpression(Optional groupExpression) { - return new LogicalFileScan(id, (ExternalTable) table, qualifier, groupExpression, + return new LogicalFileScan(relationId, (ExternalTable) table, qualifier, groupExpression, Optional.of(getLogicalProperties()), conjuncts); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalFileScan(id, (ExternalTable) table, qualifier, groupExpression, logicalProperties, conjuncts); + return new LogicalFileScan(relationId, (ExternalTable) table, qualifier, + groupExpression, logicalProperties, conjuncts); } public LogicalFileScan withConjuncts(Set conjuncts) { - return new LogicalFileScan(id, (ExternalTable) table, qualifier, groupExpression, - Optional.of(getLogicalProperties()), conjuncts); + return new LogicalFileScan(relationId, (ExternalTable) table, qualifier, groupExpression, + Optional.of(getLogicalProperties()), conjuncts); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java index 6943062e7dbd34..a1817555436dbe 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFilter.java @@ -67,7 +67,7 @@ public List getExpressions() { } @Override - public List extraPlans() { + public List extraPlans() { return conjuncts.stream().map(Expression::children).flatMap(Collection::stream).flatMap(m -> { if (m instanceof SubqueryExpr) { return Stream.of(new LogicalSubQueryAlias<>(m.toSql(), ((SubqueryExpr) m).getQueryPlan())); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java index e8778b13d7e07a..c35879f96937f6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalGenerate.java @@ -30,6 +30,7 @@ import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.Lists; import java.util.List; import java.util.Objects; @@ -79,8 +80,16 @@ public List getExpressions() { return generators; } + /** + * update generators + */ public LogicalGenerate withGenerators(List generators) { - return new LogicalGenerate<>(generators, generatorOutput, + Preconditions.checkArgument(generators.size() == generatorOutput.size()); + List newGeneratorOutput = Lists.newArrayList(); + for (int i = 0; i < generators.size(); i++) { + newGeneratorOutput.add(generatorOutput.get(i).withNullable(generators.get(i).nullable())); + } + return new LogicalGenerate<>(generators, newGeneratorOutput, Optional.empty(), Optional.of(getLogicalProperties()), child()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJdbcScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJdbcScan.java index 358712c094805e..8b3df69d16cf04 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJdbcScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJdbcScan.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -40,14 +41,14 @@ /** * Logical scan for external jdbc catalog and jdbc table. */ -public class LogicalJdbcScan extends LogicalRelation { +public class LogicalJdbcScan extends LogicalCatalogRelation { private final Set conjuncts; /** * Constructor for LogicalJdbcScan. */ - public LogicalJdbcScan(ObjectId id, TableIf table, List qualifier, + public LogicalJdbcScan(RelationId id, TableIf table, List qualifier, Optional groupExpression, Optional logicalProperties, Set conjuncts) { @@ -56,8 +57,8 @@ public LogicalJdbcScan(ObjectId id, TableIf table, List qualifier, this.conjuncts = ImmutableSet.copyOf(Objects.requireNonNull(conjuncts, "conjuncts should not be null")); } - public LogicalJdbcScan(ObjectId id, TableIf table, List qualifier) { - this(id, table, qualifier, Optional.empty(), Optional.empty(), ImmutableSet.of()); + public LogicalJdbcScan(RelationId id, TableIf table, List qualifier) { + this(id, table, qualifier, Optional.empty(), Optional.empty(),ImmutableSet.of()); } @Override @@ -77,19 +78,19 @@ public String toString() { @Override public LogicalJdbcScan withGroupExpression(Optional groupExpression) { - return new LogicalJdbcScan(id, table, qualifier, groupExpression, + return new LogicalJdbcScan(relationId, table, qualifier, groupExpression, Optional.of(getLogicalProperties()), conjuncts); } public LogicalJdbcScan withConjuncts(Set conjuncts) { - return new LogicalJdbcScan(id, table, qualifier, groupExpression, + return new LogicalJdbcScan(relationId, table, qualifier, groupExpression, Optional.of(getLogicalProperties()), conjuncts); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalJdbcScan(id, table, qualifier, groupExpression, logicalProperties, conjuncts); + return new LogicalJdbcScan(relationId, table, qualifier, groupExpression, logicalProperties, conjuncts); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java index ffd69a94e5a0ec..6d58ba1f71ce7e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapScan.java @@ -18,22 +18,18 @@ package org.apache.doris.nereids.trees.plans.logical; import org.apache.doris.catalog.Column; -import org.apache.doris.catalog.Database; -import org.apache.doris.catalog.Env; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Table; import org.apache.doris.common.util.Util; -import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.rules.rewrite.mv.AbstractSelectMaterializedIndexRule; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.PreAggStatus; -import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.OlapScan; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -55,7 +51,7 @@ /** * Logical OlapScan. */ -public class LogicalOlapScan extends LogicalRelation implements CatalogRelation, OlapScan { +public class LogicalOlapScan extends LogicalCatalogRelation implements OlapScan { /////////////////////////////////////////////////////////////////////////// // Members for materialized index. @@ -77,10 +73,10 @@ public class LogicalOlapScan extends LogicalRelation implements CatalogRelation, private final PreAggStatus preAggStatus; /** - * When the Slotreference is generated through fromColumn, + * When the SlotReference is generated through fromColumn, * the exprId will be generated incrementally, * causing the slotId of the base to change when the output is recalculated. - * This structure is responsible for storing the generated Slotreference + * This structure is responsible for storing the generated SlotReference */ private final Map cacheSlotWithSlotName; @@ -110,45 +106,40 @@ public class LogicalOlapScan extends LogicalRelation implements CatalogRelation, /////////////////////////////////////////////////////////////////////////// private final List hints; - public LogicalOlapScan(ObjectId id, OlapTable table) { + public LogicalOlapScan(RelationId id, OlapTable table) { this(id, table, ImmutableList.of()); } - public LogicalOlapScan(ObjectId id, OlapTable table, List qualifier) { + public LogicalOlapScan(RelationId id, OlapTable table, List qualifier) { this(id, table, qualifier, Optional.empty(), Optional.empty(), table.getPartitionIds(), false, ImmutableList.of(), -1, false, PreAggStatus.on(), ImmutableList.of(), ImmutableList.of(), Maps.newHashMap()); } - public LogicalOlapScan(ObjectId id, OlapTable table, List qualifier, List hints) { + public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List hints) { this(id, table, qualifier, Optional.empty(), Optional.empty(), table.getPartitionIds(), false, ImmutableList.of(), -1, false, PreAggStatus.on(), ImmutableList.of(), hints, Maps.newHashMap()); } - public LogicalOlapScan(ObjectId id, OlapTable table, List qualifier, List specifiedPartitions, + public LogicalOlapScan(RelationId id, OlapTable table, List qualifier, List specifiedPartitions, List hints) { this(id, table, qualifier, Optional.empty(), Optional.empty(), + // must use specifiedPartitions here for prune partition by sql like 'select * from t partition p1' specifiedPartitions, false, ImmutableList.of(), -1, false, PreAggStatus.on(), specifiedPartitions, hints, Maps.newHashMap()); } - public LogicalOlapScan(ObjectId id, Table table, List qualifier) { - this(id, table, qualifier, Optional.empty(), Optional.empty(), - ((OlapTable) table).getPartitionIds(), false, ImmutableList.of(), - -1, false, PreAggStatus.on(), ImmutableList.of(), ImmutableList.of(), Maps.newHashMap()); - } - /** * Constructor for LogicalOlapScan. */ - public LogicalOlapScan(ObjectId id, Table table, List qualifier, + public LogicalOlapScan(RelationId id, Table table, List qualifier, Optional groupExpression, Optional logicalProperties, List selectedPartitionIds, boolean partitionPruned, List selectedTabletIds, long selectedIndexId, boolean indexSelected, - PreAggStatus preAggStatus, List partitions, + PreAggStatus preAggStatus, List specifiedPartitions, List hints, Map cacheSlotWithSlotName) { super(id, PlanType.LOGICAL_OLAP_SCAN, table, qualifier, @@ -159,10 +150,11 @@ public LogicalOlapScan(ObjectId id, Table table, List qualifier, this.selectedIndexId = selectedIndexId <= 0 ? getTable().getBaseIndexId() : selectedIndexId; this.indexSelected = indexSelected; this.preAggStatus = preAggStatus; - this.manuallySpecifiedPartitions = ImmutableList.copyOf(partitions); + this.manuallySpecifiedPartitions = ImmutableList.copyOf(specifiedPartitions); this.selectedPartitionIds = selectedPartitionIds.stream() - .filter(partitionId -> this.getTable().getPartition(partitionId).hasData()).collect( - Collectors.toList()); + .filter(partitionId -> this.getTable().getPartition(partitionId) != null) + .filter(partitionId -> this.getTable().getPartition(partitionId).hasData()) + .collect(Collectors.toList()); this.hints = Objects.requireNonNull(hints, "hints can not be null"); this.cacheSlotWithSlotName = Objects.requireNonNull(cacheSlotWithSlotName, "mvNameToSlot can not be null"); } @@ -177,13 +169,6 @@ public OlapTable getTable() { return (OlapTable) table; } - @Override - public Database getDatabase() throws AnalysisException { - Preconditions.checkArgument(!qualifier.isEmpty()); - return Env.getCurrentInternalCatalog().getDbOrException(qualifier.get(0), - s -> new AnalysisException("Database [" + qualifier.get(0) + "] does not exist.")); - } - @Override public String toString() { return Utils.toSqlString("LogicalOlapScan", @@ -199,31 +184,31 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass() || !super.equals(o)) { + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { return false; } - return Objects.equals(id, ((LogicalOlapScan) o).id) - && Objects.equals(selectedPartitionIds, ((LogicalOlapScan) o).selectedPartitionIds) - && Objects.equals(partitionPruned, ((LogicalOlapScan) o).partitionPruned) - && Objects.equals(selectedIndexId, ((LogicalOlapScan) o).selectedIndexId) - && Objects.equals(indexSelected, ((LogicalOlapScan) o).indexSelected) - && Objects.equals(selectedTabletIds, ((LogicalOlapScan) o).selectedTabletIds) - && Objects.equals(hints, ((LogicalOlapScan) o).hints) - && Objects.equals(cacheSlotWithSlotName, ((LogicalOlapScan) o).cacheSlotWithSlotName); + LogicalOlapScan that = (LogicalOlapScan) o; + return selectedIndexId == that.selectedIndexId && indexSelected == that.indexSelected + && partitionPruned == that.partitionPruned && Objects.equals(preAggStatus, that.preAggStatus) + && Objects.equals(selectedTabletIds, that.selectedTabletIds) + && Objects.equals(manuallySpecifiedPartitions, that.manuallySpecifiedPartitions) + && Objects.equals(selectedPartitionIds, that.selectedPartitionIds) + && Objects.equals(hints, that.hints); } @Override public int hashCode() { - return Objects.hash(id, - selectedPartitionIds, partitionPruned, - selectedIndexId, indexSelected, - selectedTabletIds, - hints); + return Objects.hash(super.hashCode(), selectedIndexId, indexSelected, preAggStatus, cacheSlotWithSlotName, + selectedTabletIds, partitionPruned, manuallySpecifiedPartitions, selectedPartitionIds, hints); } @Override public LogicalOlapScan withGroupExpression(Optional groupExpression) { - return new LogicalOlapScan(id, (Table) table, qualifier, groupExpression, Optional.of(getLogicalProperties()), + return new LogicalOlapScan(relationId, (Table) table, qualifier, + groupExpression, Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, hints, cacheSlotWithSlotName); @@ -232,34 +217,38 @@ public LogicalOlapScan withGroupExpression(Optional groupExpres @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalOlapScan(id, (Table) table, qualifier, groupExpression, logicalProperties, + return new LogicalOlapScan(relationId, (Table) table, qualifier, groupExpression, logicalProperties, selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, hints, cacheSlotWithSlotName); } public LogicalOlapScan withSelectedPartitionIds(List selectedPartitionIds) { - return new LogicalOlapScan(id, (Table) table, qualifier, Optional.empty(), Optional.of(getLogicalProperties()), + return new LogicalOlapScan(relationId, (Table) table, qualifier, + Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, true, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, hints, cacheSlotWithSlotName); } public LogicalOlapScan withMaterializedIndexSelected(PreAggStatus preAgg, long indexId) { - return new LogicalOlapScan(id, (Table) table, qualifier, Optional.empty(), Optional.of(getLogicalProperties()), + return new LogicalOlapScan(relationId, (Table) table, qualifier, + Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, indexId, true, preAgg, manuallySpecifiedPartitions, hints, cacheSlotWithSlotName); } public LogicalOlapScan withSelectedTabletIds(List selectedTabletIds) { - return new LogicalOlapScan(id, (Table) table, qualifier, Optional.empty(), Optional.of(getLogicalProperties()), + return new LogicalOlapScan(relationId, (Table) table, qualifier, + Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, hints, cacheSlotWithSlotName); } public LogicalOlapScan withPreAggStatus(PreAggStatus preAggStatus) { - return new LogicalOlapScan(id, (Table) table, qualifier, Optional.empty(), Optional.of(getLogicalProperties()), + return new LogicalOlapScan(relationId, (Table) table, qualifier, + Optional.empty(), Optional.of(getLogicalProperties()), selectedPartitionIds, partitionPruned, selectedTabletIds, selectedIndexId, indexSelected, preAggStatus, manuallySpecifiedPartitions, hints, cacheSlotWithSlotName); 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 575c13b75b56e6..8a03ae08822a97 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 @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.OneRowRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -40,18 +41,17 @@ * A relation that contains only one row consist of some constant expressions. * e.g. select 100, 'value' */ -public class LogicalOneRowRelation extends LogicalLeaf implements OneRowRelation, OutputPrunable { +public class LogicalOneRowRelation extends LogicalRelation implements OneRowRelation, OutputPrunable { private final List projects; - public LogicalOneRowRelation(List projects) { - this(projects, Optional.empty(), Optional.empty()); + public LogicalOneRowRelation(RelationId relationId, List projects) { + this(relationId, projects, Optional.empty(), Optional.empty()); } - private LogicalOneRowRelation(List projects, - Optional groupExpression, - Optional logicalProperties) { - super(PlanType.LOGICAL_ONE_ROW_RELATION, groupExpression, logicalProperties); + private LogicalOneRowRelation(RelationId relationId, List projects, + Optional groupExpression, Optional logicalProperties) { + super(relationId, PlanType.LOGICAL_ONE_ROW_RELATION, groupExpression, logicalProperties); Preconditions.checkArgument(projects.stream().noneMatch(p -> p.containsType(Slot.class)), "OneRowRelation can not contains any slot"); Preconditions.checkArgument(projects.stream().noneMatch(p -> p.containsType(AggregateFunction.class)), @@ -76,13 +76,13 @@ public List getExpressions() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new LogicalOneRowRelation(projects, groupExpression, Optional.of(getLogicalProperties())); + return new LogicalOneRowRelation(relationId, projects, groupExpression, Optional.of(getLogicalProperties())); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalOneRowRelation(projects, groupExpression, logicalProperties); + return new LogicalOneRowRelation(relationId, projects, groupExpression, logicalProperties); } @Override @@ -92,13 +92,6 @@ public List computeOutput() { .collect(ImmutableList.toImmutableList()); } - @Override - public String toString() { - return Utils.toSqlString("LogicalOneRowRelation", - "projects", projects - ); - } - @Override public boolean equals(Object o) { if (this == o) { @@ -116,11 +109,18 @@ public boolean equals(Object o) { @Override public int hashCode() { - return Objects.hash(projects); + return Objects.hash(super.hashCode(), projects); + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalOneRowRelation", + "projects", projects + ); } public LogicalOneRowRelation withProjects(List namedExpressions) { - return new LogicalOneRowRelation(namedExpressions, Optional.empty(), Optional.empty()); + return new LogicalOneRowRelation(relationId, namedExpressions, Optional.empty(), Optional.empty()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalPartitionTopN.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalPartitionTopN.java index 3c6da5534a3640..003b2737277e31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalPartitionTopN.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalPartitionTopN.java @@ -175,6 +175,11 @@ public List getExpressions() { .build(); } + public LogicalPartitionTopN withPartitionKeysAndOrderKeys( + List partitionKeys, List orderKeys) { + return new LogicalPartitionTopN<>(function, partitionKeys, orderKeys, hasGlobalLimit, partitionLimit, child()); + } + @Override public LogicalPartitionTopN withChildren(List children) { Preconditions.checkArgument(children.size() == 1); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRelation.java index fdf5b79999fc86..8b1a3e59879462 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRelation.java @@ -17,18 +17,13 @@ package org.apache.doris.nereids.trees.plans.logical; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.catalog.TableIf; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.PlanType; -import org.apache.doris.nereids.trees.plans.algebra.Scan; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.algebra.Relation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; -import org.apache.doris.nereids.util.Utils; import com.google.common.collect.ImmutableList; import org.json.JSONObject; @@ -40,42 +35,19 @@ /** * Logical relation plan. */ -public abstract class LogicalRelation extends LogicalLeaf implements Scan { +public abstract class LogicalRelation extends LogicalLeaf implements Relation { - protected final ObjectId id; - protected final TableIf table; - protected final ImmutableList qualifier; + protected final RelationId relationId; - public LogicalRelation(ObjectId id, PlanType type, TableIf table, List qualifier) { - this(id, type, table, qualifier, Optional.empty(), Optional.empty()); + public LogicalRelation(RelationId relationId, PlanType type) { + this(relationId, type, Optional.empty(), Optional.empty()); } - public LogicalRelation(ObjectId id, PlanType type, Optional groupExpression, - Optional logicalProperties) { - this(id, type, new OlapTable(), ImmutableList.of(), groupExpression, logicalProperties); - } - - /** - * Constructor for LogicalRelationPlan. - * - * @param table Doris table - * @param qualifier qualified relation name - */ - public LogicalRelation(ObjectId id, PlanType type, TableIf table, List qualifier, + public LogicalRelation(RelationId relationId, PlanType type, Optional groupExpression, Optional logicalProperties) { super(type, groupExpression, logicalProperties); - this.id = id; - this.table = Objects.requireNonNull(table, "table can not be null"); - this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier, "qualifier can not be null")); - } + this.relationId = relationId; - @Override - public TableIf getTable() { - return table; - } - - public List getQualifier() { - return qualifier; } @Override @@ -87,22 +59,12 @@ public boolean equals(Object o) { return false; } LogicalRelation that = (LogicalRelation) o; - return this.id.equals(that.getId()) - && Objects.equals(this.table.getId(), that.table.getId()) - && Objects.equals(this.qualifier, that.qualifier); + return this.relationId.equals(that.getRelationId()); } @Override public int hashCode() { - return Objects.hash(id); - } - - @Override - public List computeOutput() { - return table.getBaseSchema() - .stream() - .map(col -> SlotReference.fromColumn(col, qualified())) - .collect(ImmutableList.toImmutableList()); + return Objects.hash(relationId); } @Override @@ -115,31 +77,15 @@ public List getExpressions() { return ImmutableList.of(); } - /** - * Full qualified name parts, i.e., concat qualifier and name into a list. - */ - public List qualified() { - return Utils.qualifiedNameParts(qualifier, table.getName()); - } - - /** - * Full qualified table name, concat qualifier and name with `.` as separator. - */ - public String qualifiedName() { - return Utils.qualifiedName(qualifier, table.getName()); - } - - public ObjectId getId() { - return id; + public RelationId getRelationId() { + return relationId; } @Override public JSONObject toJson() { JSONObject logicalRelation = super.toJson(); JSONObject properties = new JSONObject(); - properties.put("ObjectId", id.toString()); - properties.put("Table", table.toString()); - properties.put("Qualifier", qualifier.toString()); + properties.put("RelationId", relationId.toString()); logicalRelation.put("Properties", properties); return logicalRelation; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSchemaScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSchemaScan.java index 1b0e51d17b28bb..24cabfc04ee5a8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSchemaScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSchemaScan.java @@ -21,10 +21,9 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; -import org.apache.doris.nereids.trees.plans.algebra.Scan; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -34,13 +33,13 @@ /** * LogicalSchemaScan. */ -public class LogicalSchemaScan extends LogicalRelation implements Scan { +public class LogicalSchemaScan extends LogicalCatalogRelation { - public LogicalSchemaScan(ObjectId id, TableIf table, List qualifier) { + public LogicalSchemaScan(RelationId id, TableIf table, List qualifier) { super(id, PlanType.LOGICAL_SCHEMA_SCAN, table, qualifier); } - public LogicalSchemaScan(ObjectId id, TableIf table, List qualifier, + public LogicalSchemaScan(RelationId id, TableIf table, List qualifier, Optional groupExpression, Optional logicalProperties) { super(id, PlanType.LOGICAL_SCHEMA_SCAN, table, qualifier, groupExpression, logicalProperties); } @@ -57,23 +56,14 @@ public R accept(PlanVisitor visitor, C context) { @Override public Plan withGroupExpression(Optional groupExpression) { - return new LogicalSchemaScan(id, table, qualifier, groupExpression, Optional.of(getLogicalProperties())); + return new LogicalSchemaScan(relationId, table, qualifier, + groupExpression, Optional.of(getLogicalProperties())); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalSchemaScan(id, table, qualifier, groupExpression, logicalProperties); - } - - @Override - public boolean equals(Object o) { - return super.equals(o); - } - - @Override - public int hashCode() { - return super.hashCode(); + return new LogicalSchemaScan(relationId, table, qualifier, groupExpression, logicalProperties); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java index 308cd2a9e9c229..72a4996ece752b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSetOperation.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; +import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; @@ -103,25 +104,17 @@ public List computeOutput() { .collect(ImmutableList.toImmutableList()); } - public List> collectCastExpressions() { - return castCommonDataTypeOutputs(resetNullableForLeftOutputs()); + public List> collectChildrenProjections() { + return castCommonDataTypeOutputs(); } /** * Generate new output for SetOperation. */ - public List buildNewOutputs(List leftCastExpressions) { + public List buildNewOutputs() { ImmutableList.Builder newOutputs = new Builder<>(); - for (Expression expression : leftCastExpressions) { - if (expression instanceof Cast) { - Cast cast = ((Cast) expression); - newOutputs.add(new SlotReference( - cast.child().toSql(), expression.getDataType(), - cast.child().nullable())); - } else if (expression instanceof Slot) { - Slot slot = ((Slot) expression); - newOutputs.add(new SlotReference(slot.toSql(), slot.getDataType(), slot.nullable())); - } + for (Slot slot : resetNullableForLeftOutputs()) { + newOutputs.add(new SlotReference(slot.toSql(), slot.getDataType(), slot.nullable())); } return newOutputs.build(); } @@ -140,12 +133,12 @@ private List resetNullableForLeftOutputs() { return ImmutableList.copyOf(resetNullableForLeftOutputs); } - private List> castCommonDataTypeOutputs(List resetNullableForLeftOutputs) { - List newLeftOutputs = new ArrayList<>(); - List newRightOutputs = new ArrayList<>(); + private List> castCommonDataTypeOutputs() { + List newLeftOutputs = new ArrayList<>(); + List newRightOutputs = new ArrayList<>(); // Ensure that the output types of the left and right children are consistent and expand upward. - for (int i = 0; i < resetNullableForLeftOutputs.size(); ++i) { - Slot left = resetNullableForLeftOutputs.get(i); + for (int i = 0; i < child(0).getOutput().size(); ++i) { + Slot left = child(0).getOutput().get(i); Slot right = child(1).getOutput().get(i); DataType compatibleType = DataType.fromCatalogType(Type.getAssignmentCompatibleType( left.getDataType().toCatalogDataType(), @@ -153,11 +146,17 @@ private List> castCommonDataTypeOutputs(List resetNullabl false)); Expression newLeft = TypeCoercionUtils.castIfNotSameType(left, compatibleType); Expression newRight = TypeCoercionUtils.castIfNotSameType(right, compatibleType); - newLeftOutputs.add(newLeft); - newRightOutputs.add(newRight); + if (newLeft instanceof Cast) { + newLeft = new Alias(newLeft, left.getName()); + } + if (newRight instanceof Cast) { + newRight = new Alias(newRight, right.getName()); + } + newLeftOutputs.add((NamedExpression) newLeft); + newRightOutputs.add((NamedExpression) newRight); } - List> resultExpressions = new ArrayList<>(); + List> resultExpressions = new ArrayList<>(); resultExpressions.add(newLeftOutputs); resultExpressions.add(newRightOutputs); return ImmutableList.copyOf(resultExpressions); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java index 53bbee0002ab0a..ef8588b1fe6117 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java @@ -19,10 +19,8 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; -import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; -import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -46,8 +44,6 @@ public class LogicalSubQueryAlias extends LogicalUnary< private final List qualifier; private final Optional> columnAliases; - private final CTEId cteId; - public LogicalSubQueryAlias(String tableAlias, CHILD_TYPE child) { this(ImmutableList.of(tableAlias), Optional.empty(), Optional.empty(), Optional.empty(), child); } @@ -70,24 +66,12 @@ public LogicalSubQueryAlias(List qualifier, Optional> colum super(PlanType.LOGICAL_SUBQUERY_ALIAS, groupExpression, logicalProperties, child); this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier, "qualifier is null")); this.columnAliases = columnAliases; - this.cteId = cteId(); - } - - public LogicalSubQueryAlias(List qualifier, Optional> columnAliases, - Optional groupExpression, - Optional logicalProperties, CHILD_TYPE child, CTEId cteId) { - super(PlanType.LOGICAL_SUBQUERY_ALIAS, groupExpression, logicalProperties, child); - this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier)); - this.columnAliases = columnAliases; - this.cteId = cteId; } @Override public List computeOutput() { List childOutput = child().getOutput(); - List columnAliases = this.columnAliases.isPresent() - ? this.columnAliases.get() - : ImmutableList.of(); + List columnAliases = this.columnAliases.orElseGet(ImmutableList::of); ImmutableList.Builder currentOutput = ImmutableList.builder(); for (int i = 0; i < childOutput.size(); i++) { Slot originSlot = childOutput.get(i); @@ -115,15 +99,12 @@ public Optional> getColumnAliases() { @Override public String toString() { - if (columnAliases.isPresent()) { - return Utils.toSqlString("LogicalSubQueryAlias", - "qualifier", qualifier, - "columnAliases", StringUtils.join(columnAliases.get(), ",") - ); - } - return Utils.toSqlString("LogicalSubQueryAlias", + return columnAliases.map(strings -> Utils.toSqlString("LogicalSubQueryAlias", + "qualifier", qualifier, + "columnAliases", StringUtils.join(strings, ",") + )).orElseGet(() -> Utils.toSqlString("LogicalSubQueryAlias", "qualifier", qualifier - ); + )); } @Override @@ -172,12 +153,4 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr return new LogicalSubQueryAlias<>(qualifier, columnAliases, groupExpression, logicalProperties, children.get(0)); } - - public CTEId cteId() { - return StatementScopeIdGenerator.newCTEId(); - } - - public CTEId getCteId() { - return cteId; - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTVFRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTVFRelation.java index e068048d289f12..6c5f554b2f8320 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTVFRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalTVFRelation.java @@ -19,10 +19,12 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.TVFRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -38,41 +40,64 @@ public class LogicalTVFRelation extends LogicalRelation implements TVFRelation { private final TableValuedFunction function; - public LogicalTVFRelation(ObjectId id, TableValuedFunction function) { - super(id, PlanType.LOGICAL_TVF_RELATION, - Objects.requireNonNull(function, "table valued function can not be null").getTable(), - ImmutableList.of()); + public LogicalTVFRelation(RelationId id, TableValuedFunction function) { + super(id, PlanType.LOGICAL_TVF_RELATION); this.function = function; } - public LogicalTVFRelation(ObjectId id, TableValuedFunction function, Optional groupExpression, + public LogicalTVFRelation(RelationId id, TableValuedFunction function, Optional groupExpression, Optional logicalProperties) { - super(id, PlanType.LOGICAL_TVF_RELATION, - Objects.requireNonNull(function, "table valued function can not be null").getTable(), - ImmutableList.of(), groupExpression, logicalProperties); + super(id, PlanType.LOGICAL_TVF_RELATION, groupExpression, logicalProperties); this.function = function; } @Override public LogicalTVFRelation withGroupExpression(Optional groupExpression) { - return new LogicalTVFRelation(id, function, groupExpression, Optional.of(getLogicalProperties())); + return new LogicalTVFRelation(relationId, function, groupExpression, Optional.of(getLogicalProperties())); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalTVFRelation(id, function, groupExpression, logicalProperties); + return new LogicalTVFRelation(relationId, function, groupExpression, logicalProperties); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + LogicalTVFRelation that = (LogicalTVFRelation) o; + return Objects.equals(function, that.function); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), function); } @Override public String toString() { return Utils.toSqlString("LogicalTVFRelation", - "qualified", qualifiedName(), "output", getOutput(), "function", function.toSql() ); } + @Override + public List computeOutput() { + return function.getTable().getBaseSchema() + .stream() + .map(col -> SlotReference.fromColumn(col, ImmutableList.of())) + .collect(ImmutableList.toImmutableList()); + } + @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitLogicalTVFRelation(this, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEAnchor.java index 7c6ee90585e256..88908abe30ee81 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEAnchor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEAnchor.java @@ -46,22 +46,18 @@ public class PhysicalCTEAnchor< private final CTEId cteId; public PhysicalCTEAnchor(CTEId cteId, LogicalProperties logicalProperties, - LEFT_CHILD_TYPE leftChild, - RIGHT_CHILD_TYPE rightChild) { + LEFT_CHILD_TYPE leftChild, RIGHT_CHILD_TYPE rightChild) { this(cteId, Optional.empty(), logicalProperties, leftChild, rightChild); } public PhysicalCTEAnchor(CTEId cteId, Optional groupExpression, - LogicalProperties logicalProperties, - LEFT_CHILD_TYPE leftChild, - RIGHT_CHILD_TYPE rightChild) { + LogicalProperties logicalProperties, LEFT_CHILD_TYPE leftChild, RIGHT_CHILD_TYPE rightChild) { this(cteId, groupExpression, logicalProperties, null, null, leftChild, rightChild); } public PhysicalCTEAnchor(CTEId cteId, Optional groupExpression, - LogicalProperties logicalProperties, PhysicalProperties physicalProperties, - Statistics statistics, LEFT_CHILD_TYPE leftChild, - RIGHT_CHILD_TYPE rightChild) { + LogicalProperties logicalProperties, PhysicalProperties physicalProperties, + Statistics statistics, LEFT_CHILD_TYPE leftChild, RIGHT_CHILD_TYPE rightChild) { super(PlanType.PHYSICAL_CTE_ANCHOR, groupExpression, logicalProperties, physicalProperties, statistics, leftChild, rightChild); this.cteId = cteId; @@ -109,7 +105,7 @@ public R accept(PlanVisitor visitor, C context) { public PhysicalCTEAnchor withChildren(List children) { Preconditions.checkArgument(children.size() == 2); return new PhysicalCTEAnchor<>(cteId, groupExpression, getLogicalProperties(), physicalProperties, - statistics, children.get(0), children.get(1)); + statistics, children.get(0), children.get(1)); } @Override @@ -127,14 +123,15 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr @Override public PhysicalCTEAnchor withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, - Statistics statistics) { + Statistics statistics) { return new PhysicalCTEAnchor<>(cteId, groupExpression, getLogicalProperties(), physicalProperties, - statistics, child(0), child(1)); + statistics, child(0), child(1)); } @Override public String shapeInfo() { - return Utils.toSqlString("CteAnchor[cteId=", cteId, "]"); + return Utils.toSqlString("PhysicalCteAnchor", + "cteId", cteId); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java index 91c0f2a4034a3a..e6450a94e8e0ff 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java @@ -17,23 +17,19 @@ package org.apache.doris.nereids.trees.plans.physical; -import org.apache.doris.catalog.OlapTable; -import org.apache.doris.nereids.exceptions.TransformException; 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.CTEId; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import java.util.List; @@ -53,48 +49,35 @@ public class PhysicalCTEConsumer extends PhysicalRelation { /** * Constructor */ - public PhysicalCTEConsumer(CTEId cteId, Map consumerToProducerSlotMap, - Map producerToConsumerSlotMap, - LogicalProperties logicalProperties) { - this(cteId, consumerToProducerSlotMap, producerToConsumerSlotMap, + public PhysicalCTEConsumer(RelationId relationId, CTEId cteId, Map consumerToProducerSlotMap, + Map producerToConsumerSlotMap, LogicalProperties logicalProperties) { + this(relationId, cteId, consumerToProducerSlotMap, producerToConsumerSlotMap, Optional.empty(), logicalProperties); } /** * Constructor */ - public PhysicalCTEConsumer(CTEId cteId, Map consumerToProducerSlotMap, - Map producerToConsumerSlotMap, - Optional groupExpression, - LogicalProperties logicalProperties) { - this(cteId, consumerToProducerSlotMap, producerToConsumerSlotMap, groupExpression, logicalProperties, - null, null); + public PhysicalCTEConsumer(RelationId relationId, CTEId cteId, + Map consumerToProducerSlotMap, Map producerToConsumerSlotMap, + Optional groupExpression, LogicalProperties logicalProperties) { + this(relationId, cteId, consumerToProducerSlotMap, producerToConsumerSlotMap, + groupExpression, logicalProperties, null, null); } /** * Constructor */ - public PhysicalCTEConsumer(CTEId cteId, Map consumerToProducerSlotMap, - Map producerToConsumerSlotMap, - Optional groupExpression, - LogicalProperties logicalProperties, - PhysicalProperties physicalProperties, - Statistics statistics) { - super(RelationUtil.newRelationId(), PlanType.PHYSICAL_CTE_CONSUME, ImmutableList.of(), groupExpression, + public PhysicalCTEConsumer(RelationId relationId, CTEId cteId, Map consumerToProducerSlotMap, + Map producerToConsumerSlotMap, Optional groupExpression, + LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { + super(relationId, PlanType.PHYSICAL_CTE_CONSUME, groupExpression, logicalProperties, physicalProperties, statistics); this.cteId = cteId; - this.consumerToProducerSlotMap = ImmutableMap.copyOf(consumerToProducerSlotMap); - this.producerToConsumerSlotMap = ImmutableMap.copyOf(producerToConsumerSlotMap); - } - - @Override - public OlapTable getTable() { - throw new TransformException("should not reach here"); - } - - @Override - public List getQualifier() { - throw new TransformException("should not reach here"); + this.consumerToProducerSlotMap = ImmutableMap.copyOf(Objects.requireNonNull( + consumerToProducerSlotMap, "consumerToProducerSlotMap should not null")); + this.producerToConsumerSlotMap = ImmutableMap.copyOf(Objects.requireNonNull( + producerToConsumerSlotMap, "consumerToProducerSlotMap should not null")); } public CTEId getCteId() { @@ -105,35 +88,17 @@ public Map getProducerToConsumerSlotMap() { return producerToConsumerSlotMap; } - @Override - public List getExpressions() { - return ImmutableList.of(); - } - - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - - if (!super.equals(o)) { - return false; - } - - PhysicalCTEConsumer that = (PhysicalCTEConsumer) o; - return Objects.equals(cteId, that.cteId) - && Objects.equals(producerToConsumerSlotMap, that.producerToConsumerSlotMap) - && Objects.equals(consumerToProducerSlotMap, that.consumerToProducerSlotMap); - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), cteId, producerToConsumerSlotMap, consumerToProducerSlotMap); + public Slot getProducerSlot(Slot consumerSlot) { + Slot slot = consumerToProducerSlotMap.get(consumerSlot); + Preconditions.checkArgument(slot != null, String.format( + "Required producer slot for %s doesn't exist", consumerSlot)); + return slot; } @Override public String toString() { - return Utils.toSqlString("PhysicalCTEConsumer", "cteId", cteId); + return Utils.toSqlString("PhysicalCTEConsumer[" + id.asInt() + "]", + "cteId", cteId); } @Override @@ -141,42 +106,32 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitPhysicalCTEConsumer(this, context); } - @Override - public PhysicalCTEConsumer withChildren(List children) { - Preconditions.checkArgument(children.isEmpty()); - return new PhysicalCTEConsumer(cteId, consumerToProducerSlotMap, producerToConsumerSlotMap, - getLogicalProperties()); - } - @Override public PhysicalCTEConsumer withGroupExpression(Optional groupExpression) { - return new PhysicalCTEConsumer(cteId, consumerToProducerSlotMap, producerToConsumerSlotMap, + return new PhysicalCTEConsumer(relationId, cteId, + consumerToProducerSlotMap, producerToConsumerSlotMap, groupExpression, getLogicalProperties()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalCTEConsumer(cteId, consumerToProducerSlotMap, producerToConsumerSlotMap, + return new PhysicalCTEConsumer(relationId, cteId, + consumerToProducerSlotMap, producerToConsumerSlotMap, groupExpression, logicalProperties.get()); } @Override public PhysicalCTEConsumer withPhysicalPropertiesAndStats( PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalCTEConsumer(cteId, consumerToProducerSlotMap, producerToConsumerSlotMap, + return new PhysicalCTEConsumer(relationId, cteId, + consumerToProducerSlotMap, producerToConsumerSlotMap, groupExpression, getLogicalProperties(), physicalProperties, statistics); } @Override public String shapeInfo() { - return Utils.toSqlString("CteConsumer[cteId=", cteId, "]"); - } - - public Slot findProducerSlot(Slot consumerSlot) { - Slot slot = consumerToProducerSlotMap.get(consumerSlot); - Preconditions.checkArgument(slot != null, String.format("Required producer" - + "slot for :%s doesn't exist", consumerSlot)); - return slot; + return Utils.toSqlString("PhysicalCteConsumer", + "cteId", cteId); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java index 4a7879246f2cd0..385aa9e1d33db1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java @@ -22,7 +22,6 @@ import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -42,35 +41,27 @@ public class PhysicalCTEProducer extends PhysicalUnary { private final CTEId cteId; - private final List projects; - public PhysicalCTEProducer(CTEId cteId, List projects, - LogicalProperties logicalProperties, CHILD_TYPE child) { - this(cteId, projects, Optional.empty(), logicalProperties, child); + public PhysicalCTEProducer(CTEId cteId, LogicalProperties logicalProperties, CHILD_TYPE child) { + this(cteId, Optional.empty(), logicalProperties, child); } - public PhysicalCTEProducer(CTEId cteId, List projects, - Optional groupExpression, + public PhysicalCTEProducer(CTEId cteId, Optional groupExpression, LogicalProperties logicalProperties, CHILD_TYPE child) { - this(cteId, projects, groupExpression, logicalProperties, null, null, child); + this(cteId, groupExpression, logicalProperties, null, null, child); } - public PhysicalCTEProducer(CTEId cteId, List projects, Optional groupExpression, - LogicalProperties logicalProperties, PhysicalProperties physicalProperties, - Statistics statistics, CHILD_TYPE child) { + public PhysicalCTEProducer(CTEId cteId, Optional groupExpression, + LogicalProperties logicalProperties, PhysicalProperties physicalProperties, + Statistics statistics, CHILD_TYPE child) { super(PlanType.PHYSICAL_CTE_PRODUCE, groupExpression, logicalProperties, physicalProperties, statistics, child); this.cteId = cteId; - this.projects = ImmutableList.copyOf(projects); } public CTEId getCteId() { return cteId; } - public List getProjects() { - return projects; - } - @Override public List getExpressions() { return ImmutableList.of(); @@ -97,7 +88,8 @@ public int hashCode() { @Override public String toString() { - return Utils.toSqlString("PhysicalCTEProducer", "cteId", cteId); + return Utils.toSqlString("PhysicalCTEProducer[" + id.asInt() + "]", + "cteId", cteId); } @Override @@ -108,30 +100,32 @@ public R accept(PlanVisitor visitor, C context) { @Override public PhysicalCTEProducer withChildren(List children) { Preconditions.checkArgument(children.size() == 1); - return new PhysicalCTEProducer<>(cteId, projects, getLogicalProperties(), children.get(0)); + return new PhysicalCTEProducer<>(cteId, groupExpression, + getLogicalProperties(), physicalProperties, statistics, children.get(0)); } @Override public PhysicalCTEProducer withGroupExpression(Optional groupExpression) { - return new PhysicalCTEProducer<>(cteId, projects, groupExpression, getLogicalProperties(), child()); + return new PhysicalCTEProducer<>(cteId, groupExpression, getLogicalProperties(), child()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalCTEProducer<>(cteId, projects, groupExpression, logicalProperties.get(), children.get(0)); + return new PhysicalCTEProducer<>(cteId, groupExpression, logicalProperties.get(), children.get(0)); } @Override public PhysicalCTEProducer withPhysicalPropertiesAndStats( PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalCTEProducer<>(cteId, projects, groupExpression, getLogicalProperties(), physicalProperties, + return new PhysicalCTEProducer<>(cteId, groupExpression, getLogicalProperties(), physicalProperties, statistics, child()); } @Override public String shapeInfo() { - return Utils.toSqlString("CteProducer[cteId=", cteId, "]"); + return Utils.toSqlString("PhysicalCteProducer", + "cteId", cteId); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCatalogRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCatalogRelation.java new file mode 100644 index 00000000000000..b25a7c530afb85 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCatalogRelation.java @@ -0,0 +1,116 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.physical; + +import org.apache.doris.catalog.Database; +import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.nereids.exceptions.AnalysisException; +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.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.Statistics; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * relation generated from TableIf + */ +public abstract class PhysicalCatalogRelation extends PhysicalRelation implements CatalogRelation { + + protected final TableIf table; + protected final ImmutableList qualifier; + + /** + * Constructor for PhysicalCatalogRelation. + * + * @param table Doris table + * @param qualifier qualified relation name + */ + public PhysicalCatalogRelation(RelationId relationId, PlanType type, TableIf table, List qualifier, + Optional groupExpression, LogicalProperties logicalProperties) { + super(relationId, type, groupExpression, logicalProperties); + this.table = Objects.requireNonNull(table, "table can not be null"); + this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier, "qualifier can not be null")); + } + + /** + * Constructor for PhysicalCatalogRelation. + * + * @param table Doris table + * @param qualifier qualified relation name + */ + public PhysicalCatalogRelation(RelationId relationId, PlanType type, TableIf table, List qualifier, + Optional groupExpression, LogicalProperties logicalProperties, + PhysicalProperties physicalProperties, + Statistics statistics) { + super(relationId, type, groupExpression, logicalProperties, physicalProperties, statistics); + this.table = Objects.requireNonNull(table, "table can not be null"); + this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier, "qualifier can not be null")); + } + + @Override + public TableIf getTable() { + return table; + } + + @Override + public Database getDatabase() throws AnalysisException { + Preconditions.checkArgument(!qualifier.isEmpty()); + return Env.getCurrentInternalCatalog().getDbOrException(qualifier.get(0), + s -> new AnalysisException("Database [" + qualifier.get(0) + "] does not exist.")); + } + + @Override + public List computeOutput() { + return table.getBaseSchema() + .stream() + .map(col -> SlotReference.fromColumn(col, qualified())) + .collect(ImmutableList.toImmutableList()); + } + + public List getQualifier() { + return qualifier; + } + + /** + * Full qualified name parts, i.e., concat qualifier and name into a list. + */ + public List qualified() { + return Utils.qualifiedNameParts(qualifier, table.getName()); + } + + /** + * Full qualified table name, concat qualifier and name with `.` as separator. + */ + public String qualifiedName() { + return Utils.qualifiedName(qualifier, table.getName()); + } + +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalEmptyRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalEmptyRelation.java index d0865b83b8a995..2a9c344bd4681f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalEmptyRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalEmptyRelation.java @@ -20,11 +20,11 @@ import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; -import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.EmptyRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -41,19 +41,21 @@ * e.g. * select * from tbl limit 0 */ -public class PhysicalEmptyRelation extends PhysicalLeaf implements EmptyRelation { +public class PhysicalEmptyRelation extends PhysicalRelation implements EmptyRelation { private final List projects; - public PhysicalEmptyRelation(List projects, LogicalProperties logicalProperties) { - this(projects, Optional.empty(), logicalProperties, null, null); + public PhysicalEmptyRelation(RelationId relationId, List projects, + LogicalProperties logicalProperties) { + this(relationId, projects, Optional.empty(), logicalProperties, null, null); } - public PhysicalEmptyRelation(List projects, Optional groupExpression, + public PhysicalEmptyRelation(RelationId relationId, List projects, + Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { - super(PlanType.PHYSICAL_EMPTY_RELATION, groupExpression, logicalProperties, physicalProperties, - statistics); + super(relationId, PlanType.PHYSICAL_EMPTY_RELATION, groupExpression, + logicalProperties, physicalProperties, statistics); this.projects = ImmutableList.copyOf(Objects.requireNonNull(projects, "projects can not be null")); } @@ -62,21 +64,16 @@ public R accept(PlanVisitor visitor, C context) { return visitor.visitPhysicalEmptyRelation(this, context); } - @Override - public List getExpressions() { - return ImmutableList.of(); - } - @Override public Plan withGroupExpression(Optional groupExpression) { - return new PhysicalEmptyRelation(projects, groupExpression, + return new PhysicalEmptyRelation(relationId, projects, groupExpression, logicalPropertiesSupplier.get(), physicalProperties, statistics); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalEmptyRelation(projects, groupExpression, + return new PhysicalEmptyRelation(relationId, projects, groupExpression, logicalProperties.get(), physicalProperties, statistics); } @@ -94,26 +91,6 @@ public String toString() { ); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - PhysicalEmptyRelation that = (PhysicalEmptyRelation) o; - return Objects.equals(projects, that.projects); - } - - @Override - public int hashCode() { - return Objects.hash(projects); - } - @Override public List getProjects() { return projects; @@ -122,7 +99,7 @@ public List getProjects() { @Override public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalEmptyRelation(projects, Optional.empty(), + return new PhysicalEmptyRelation(relationId, projects, Optional.empty(), logicalPropertiesSupplier.get(), physicalProperties, statistics); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalEsScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalEsScan.java index 2dcc0bd0ccd6d8..4ee49ca198c373 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalEsScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalEsScan.java @@ -22,46 +22,41 @@ import org.apache.doris.nereids.properties.DistributionSpec; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; import java.util.List; -import java.util.Objects; import java.util.Optional; /** * Physical es scan for external catalog. */ -public class PhysicalEsScan extends PhysicalRelation { +public class PhysicalEsScan extends PhysicalCatalogRelation { - private final ExternalTable table; private final DistributionSpec distributionSpec; /** * Constructor for PhysicalEsScan. */ - public PhysicalEsScan(ObjectId id, ExternalTable table, List qualifier, - DistributionSpec distributionSpec, Optional groupExpression, - LogicalProperties logicalProperties) { - super(id, PlanType.PHYSICAL_ES_SCAN, qualifier, groupExpression, logicalProperties); - this.table = table; + public PhysicalEsScan(RelationId id, ExternalTable table, List qualifier, + DistributionSpec distributionSpec, Optional groupExpression, + LogicalProperties logicalProperties) { + super(id, PlanType.PHYSICAL_ES_SCAN, table, qualifier, groupExpression, logicalProperties); this.distributionSpec = distributionSpec; } /** * Constructor for PhysicalEsScan. */ - public PhysicalEsScan(ObjectId id, ExternalTable table, List qualifier, - DistributionSpec distributionSpec, Optional groupExpression, - LogicalProperties logicalProperties, PhysicalProperties physicalProperties, - Statistics statistics) { - super(id, PlanType.PHYSICAL_ES_SCAN, qualifier, groupExpression, logicalProperties, + public PhysicalEsScan(RelationId id, ExternalTable table, List qualifier, + DistributionSpec distributionSpec, Optional groupExpression, + LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { + super(id, PlanType.PHYSICAL_ES_SCAN, table, qualifier, groupExpression, logicalProperties, physicalProperties, statistics); - this.table = table; this.distributionSpec = distributionSpec; } @@ -74,23 +69,6 @@ public String toString() { ); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass() || !super.equals(o)) { - return false; - } - PhysicalEsScan that = ((PhysicalEsScan) o); - return Objects.equals(table, that.table); - } - - @Override - public int hashCode() { - return Objects.hash(id, table); - } - @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitPhysicalEsScan(this, context); @@ -98,24 +76,26 @@ public R accept(PlanVisitor visitor, C context) { @Override public PhysicalEsScan withGroupExpression(Optional groupExpression) { - return new PhysicalEsScan(id, table, qualifier, distributionSpec, groupExpression, getLogicalProperties()); + return new PhysicalEsScan(relationId, getTable(), qualifier, distributionSpec, + groupExpression, getLogicalProperties()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalEsScan(id, table, qualifier, distributionSpec, groupExpression, logicalProperties.get()); + return new PhysicalEsScan(relationId, getTable(), qualifier, distributionSpec, + groupExpression, logicalProperties.get()); } @Override public ExternalTable getTable() { - return table; + return (ExternalTable) table; } @Override public PhysicalEsScan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statsDeriveResult) { - return new PhysicalEsScan(id, table, qualifier, distributionSpec, groupExpression, getLogicalProperties(), - physicalProperties, statsDeriveResult); + return new PhysicalEsScan(relationId, getTable(), qualifier, distributionSpec, + groupExpression, getLogicalProperties(), physicalProperties, statsDeriveResult); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFileScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFileScan.java index f9584c0c46d0db..b5af33d107942f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFileScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFileScan.java @@ -23,35 +23,32 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.Set; /** * Physical file scan for external catalog. */ -public class PhysicalFileScan extends PhysicalRelation { +public class PhysicalFileScan extends PhysicalCatalogRelation { - private final ExternalTable table; private final DistributionSpec distributionSpec; private final Set conjuncts; /** * Constructor for PhysicalFileScan. */ - public PhysicalFileScan(ObjectId id, ExternalTable table, List qualifier, - DistributionSpec distributionSpec, Optional groupExpression, - LogicalProperties logicalProperties, Set conjuncts) { - super(id, PlanType.PHYSICAL_FILE_SCAN, qualifier, groupExpression, logicalProperties); - this.table = table; + public PhysicalFileScan(RelationId id, ExternalTable table, List qualifier, + DistributionSpec distributionSpec, Optional groupExpression, + LogicalProperties logicalProperties, Set conjuncts) { + super(id, PlanType.PHYSICAL_FILE_SCAN, table, qualifier, groupExpression, logicalProperties); this.distributionSpec = distributionSpec; this.conjuncts = conjuncts; } @@ -59,13 +56,12 @@ public PhysicalFileScan(ObjectId id, ExternalTable table, List qualifier /** * Constructor for PhysicalFileScan. */ - public PhysicalFileScan(ObjectId id, ExternalTable table, List qualifier, - DistributionSpec distributionSpec, Optional groupExpression, - LogicalProperties logicalProperties, PhysicalProperties physicalProperties, - Statistics statistics, Set conjuncts) { - super(id, PlanType.PHYSICAL_FILE_SCAN, qualifier, groupExpression, logicalProperties, + public PhysicalFileScan(RelationId id, ExternalTable table, List qualifier, + DistributionSpec distributionSpec, Optional groupExpression, + LogicalProperties logicalProperties, PhysicalProperties physicalProperties, + Statistics statistics, Set conjuncts) { + super(id, PlanType.PHYSICAL_FILE_SCAN, table, qualifier, groupExpression, logicalProperties, physicalProperties, statistics); - this.table = table; this.distributionSpec = distributionSpec; this.conjuncts = conjuncts; } @@ -79,23 +75,6 @@ public String toString() { ); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass() || !super.equals(o)) { - return false; - } - PhysicalFileScan that = ((PhysicalFileScan) o); - return Objects.equals(table, that.table); - } - - @Override - public int hashCode() { - return Objects.hash(id, table); - } - @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitPhysicalFileScan(this, context); @@ -103,27 +82,27 @@ public R accept(PlanVisitor visitor, C context) { @Override public PhysicalFileScan withGroupExpression(Optional groupExpression) { - return new PhysicalFileScan(id, table, qualifier, distributionSpec, + return new PhysicalFileScan(relationId, getTable(), qualifier, distributionSpec, groupExpression, getLogicalProperties(), conjuncts); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalFileScan(id, table, qualifier, distributionSpec, + return new PhysicalFileScan(relationId, getTable(), qualifier, distributionSpec, groupExpression, logicalProperties.get(), conjuncts); } @Override public ExternalTable getTable() { - return table; + return (ExternalTable) table; } @Override public PhysicalFileScan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalFileScan(id, table, qualifier, distributionSpec, groupExpression, getLogicalProperties(), - physicalProperties, statistics, conjuncts); + return new PhysicalFileScan(relationId, getTable(), qualifier, distributionSpec, + groupExpression, getLogicalProperties(), physicalProperties, statistics, conjuncts); } public Set getConjuncts() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalJdbcScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalJdbcScan.java index ed17dd05d98d2f..d91d2b22f6c027 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalJdbcScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalJdbcScan.java @@ -19,13 +19,13 @@ import org.apache.doris.catalog.TableIf; import org.apache.doris.nereids.memo.GroupExpression; -import org.apache.doris.nereids.properties.DistributionSpec; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; @@ -33,14 +33,13 @@ import com.google.common.collect.ImmutableSet; import java.util.List; -import java.util.Objects; import java.util.Optional; import java.util.Set; /** * Physical jdbc scan for external catalog. */ -public class PhysicalJdbcScan extends PhysicalRelation { +public class PhysicalJdbcScan extends PhysicalCatalogRelation { private final TableIf table; private final DistributionSpec distributionSpec; @@ -49,7 +48,7 @@ public class PhysicalJdbcScan extends PhysicalRelation { /** * Constructor for PhysicalJdbcScan. */ - public PhysicalJdbcScan(ObjectId id, TableIf table, List qualifier, DistributionSpec distributionSpec, + public PhysicalJdbcScan(RelationId id, TableIf table, List qualifier, DistributionSpec distributionSpec, Optional groupExpression, LogicalProperties logicalProperties, Set conjuncts) { super(id, PlanType.PHYSICAL_JDBC_SCAN, qualifier, groupExpression, logicalProperties); this.table = table; @@ -60,7 +59,7 @@ public PhysicalJdbcScan(ObjectId id, TableIf table, List qualifier, Dist /** * Constructor for PhysicalJdbcScan. */ - public PhysicalJdbcScan(ObjectId id, TableIf table, List qualifier, + public PhysicalJdbcScan(RelationId id, TableIf table, List qualifier, DistributionSpec distributionSpec, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics, Set conjuncts) { @@ -80,23 +79,6 @@ public String toString() { ); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass() || !super.equals(o)) { - return false; - } - PhysicalJdbcScan that = ((PhysicalJdbcScan) o); - return Objects.equals(table, that.table); - } - - @Override - public int hashCode() { - return Objects.hash(id, table); - } - @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitPhysicalJdbcScan(this, context); @@ -104,14 +86,14 @@ public R accept(PlanVisitor visitor, C context) { @Override public PhysicalJdbcScan withGroupExpression(Optional groupExpression) { - return new PhysicalJdbcScan(id, table, qualifier, distributionSpec, + return new PhysicalJdbcScan(relationId, table, qualifier, distributionSpec, groupExpression, getLogicalProperties(), conjuncts); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalJdbcScan(id, table, qualifier, distributionSpec, + return new PhysicalJdbcScan(relationId, table, qualifier, distributionSpec, groupExpression, logicalProperties.get(), conjuncts); } @@ -123,7 +105,7 @@ public TableIf getTable() { @Override public PhysicalJdbcScan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalJdbcScan(id, table, qualifier, distributionSpec, groupExpression, + return new PhysicalJdbcScan(relationId, table, qualifier, distributionSpec, groupExpression, getLogicalProperties(), physicalProperties, statistics, conjuncts); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapScan.java index 87823d9bc7bf09..b98d30f4e04f56 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapScan.java @@ -24,10 +24,10 @@ import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.AbstractPlan; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.PreAggStatus; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.OlapScan; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -43,23 +43,21 @@ /** * Physical olap scan plan. */ -public class PhysicalOlapScan extends PhysicalRelation implements OlapScan { +public class PhysicalOlapScan extends PhysicalCatalogRelation implements OlapScan { public static final String DEFERRED_MATERIALIZED_SLOTS = "deferred_materialized_slots"; - private final OlapTable olapTable; private final DistributionSpec distributionSpec; private final long selectedIndexId; private final ImmutableList selectedTabletIds; private final ImmutableList selectedPartitionIds; private final PreAggStatus preAggStatus; - private final List baseOutputs; /** * Constructor for PhysicalOlapScan. */ - public PhysicalOlapScan(ObjectId id, OlapTable olapTable, List qualifier, long selectedIndexId, + public PhysicalOlapScan(RelationId id, OlapTable olapTable, List qualifier, long selectedIndexId, List selectedTabletIds, List selectedPartitionIds, DistributionSpec distributionSpec, PreAggStatus preAggStatus, List baseOutputs, Optional groupExpression, LogicalProperties logicalProperties) { @@ -70,14 +68,13 @@ public PhysicalOlapScan(ObjectId id, OlapTable olapTable, List qualifier /** * Constructor for PhysicalOlapScan. */ - public PhysicalOlapScan(ObjectId id, OlapTable olapTable, List qualifier, long selectedIndexId, + public PhysicalOlapScan(RelationId id, OlapTable olapTable, List qualifier, long selectedIndexId, List selectedTabletIds, List selectedPartitionIds, DistributionSpec distributionSpec, PreAggStatus preAggStatus, List baseOutputs, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { - super(id, PlanType.PHYSICAL_OLAP_SCAN, qualifier, groupExpression, logicalProperties, physicalProperties, - statistics); - this.olapTable = olapTable; + super(id, PlanType.PHYSICAL_OLAP_SCAN, olapTable, qualifier, + groupExpression, logicalProperties, physicalProperties, statistics); this.selectedIndexId = selectedIndexId; this.selectedTabletIds = ImmutableList.copyOf(selectedTabletIds); this.selectedPartitionIds = ImmutableList.copyOf(selectedPartitionIds); @@ -102,7 +99,7 @@ public List getSelectedPartitionIds() { @Override public OlapTable getTable() { - return olapTable; + return (OlapTable) table; } public DistributionSpec getDistributionSpec() { @@ -119,8 +116,8 @@ public List getBaseOutputs() { @Override public String toString() { - return Utils.toSqlString("PhysicalOlapScan[" + id.asInt() + "]" + getGroupIdAsString(), - "qualified", Utils.qualifiedName(qualifier, olapTable.getName()), + return Utils.toSqlString("PhysicalOlapScan[" + relationId.asInt() + "]" + getGroupIdAsString(), + "qualified", Utils.qualifiedName(qualifier, table.getName()), "stats", statistics, "fr", getMutableState(AbstractPlan.FRAGMENT_ID) ); } @@ -130,19 +127,25 @@ public boolean equals(Object o) { if (this == o) { return true; } - if (o == null || getClass() != o.getClass() || !super.equals(o)) { + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { return false; } - PhysicalOlapScan that = ((PhysicalOlapScan) o); - return Objects.equals(selectedIndexId, that.selectedIndexId) - && Objects.equals(selectedTabletIds, that.selectedPartitionIds) - && Objects.equals(selectedPartitionIds, that.selectedPartitionIds) - && Objects.equals(olapTable, that.olapTable); + PhysicalOlapScan olapScan = (PhysicalOlapScan) o; + return selectedIndexId == olapScan.selectedIndexId && Objects.equals(distributionSpec, + olapScan.distributionSpec) && Objects.equals(selectedTabletIds, olapScan.selectedTabletIds) + && Objects.equals(selectedPartitionIds, olapScan.selectedPartitionIds) + && Objects.equals(preAggStatus, olapScan.preAggStatus) && Objects.equals(baseOutputs, + olapScan.baseOutputs); } @Override public int hashCode() { - return Objects.hash(id, selectedIndexId, selectedPartitionIds, selectedTabletIds, olapTable); + return Objects.hash(super.hashCode(), distributionSpec, selectedIndexId, selectedTabletIds, + selectedPartitionIds, + preAggStatus, baseOutputs); } @Override @@ -152,7 +155,7 @@ public R accept(PlanVisitor visitor, C context) { @Override public PhysicalOlapScan withGroupExpression(Optional groupExpression) { - return new PhysicalOlapScan(id, olapTable, qualifier, selectedIndexId, selectedTabletIds, + return new PhysicalOlapScan(relationId, getTable(), qualifier, selectedIndexId, selectedTabletIds, selectedPartitionIds, distributionSpec, preAggStatus, baseOutputs, groupExpression, getLogicalProperties()); } @@ -160,7 +163,7 @@ public PhysicalOlapScan withGroupExpression(Optional groupExpre @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalOlapScan(id, olapTable, qualifier, selectedIndexId, selectedTabletIds, + return new PhysicalOlapScan(relationId, getTable(), qualifier, selectedIndexId, selectedTabletIds, selectedPartitionIds, distributionSpec, preAggStatus, baseOutputs, groupExpression, logicalProperties.get()); } @@ -168,23 +171,21 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr @Override public PhysicalOlapScan withPhysicalPropertiesAndStats( PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalOlapScan(id, olapTable, qualifier, selectedIndexId, selectedTabletIds, + return new PhysicalOlapScan(relationId, getTable(), qualifier, selectedIndexId, selectedTabletIds, selectedPartitionIds, distributionSpec, preAggStatus, baseOutputs, groupExpression, getLogicalProperties(), physicalProperties, statistics); } @Override public String shapeInfo() { - StringBuilder builder = new StringBuilder(); - builder.append(this.getClass().getSimpleName()).append("[").append(olapTable.getName()).append("]"); - return builder.toString(); + return this.getClass().getSimpleName() + "[" + table.getName() + "]"; } @Override public JSONObject toJson() { JSONObject olapScan = super.toJson(); JSONObject properties = new JSONObject(); - properties.put("OlapTable", olapTable.toString()); + properties.put("OlapTable", table.toString()); properties.put("DistributionSpec", distributionSpec.toString()); properties.put("SelectedIndexId", selectedIndexId); properties.put("SelectedTabletIds", selectedTabletIds.toString()); 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 00328f739edbe8..8ed08a642aa44b 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 @@ -24,6 +24,7 @@ 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.RelationId; import org.apache.doris.nereids.trees.plans.algebra.OneRowRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -39,20 +40,21 @@ * A physical relation that contains only one row consist of some constant expressions. * e.g. select 100, 'value' */ -public class PhysicalOneRowRelation extends PhysicalLeaf implements OneRowRelation { +public class PhysicalOneRowRelation extends PhysicalRelation implements OneRowRelation { private final List projects; - public PhysicalOneRowRelation(List projects, LogicalProperties logicalProperties) { - this(projects, Optional.empty(), logicalProperties, null, null); + public PhysicalOneRowRelation(RelationId relationId, List projects, + LogicalProperties logicalProperties) { + this(relationId, projects, Optional.empty(), logicalProperties, null, null); } - private PhysicalOneRowRelation(List projects, + private PhysicalOneRowRelation(RelationId relationId, List projects, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { - super(PlanType.PHYSICAL_ONE_ROW_RELATION, groupExpression, logicalProperties, physicalProperties, - statistics); + super(relationId, PlanType.PHYSICAL_ONE_ROW_RELATION, groupExpression, + logicalProperties, physicalProperties, statistics); this.projects = ImmutableList.copyOf(Objects.requireNonNull(projects, "projects can not be null")); } @@ -73,24 +75,17 @@ public List getExpressions() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new PhysicalOneRowRelation(projects, groupExpression, + return new PhysicalOneRowRelation(relationId, projects, groupExpression, logicalPropertiesSupplier.get(), physicalProperties, statistics); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalOneRowRelation(projects, groupExpression, + return new PhysicalOneRowRelation(relationId, projects, groupExpression, logicalProperties.get(), physicalProperties, statistics); } - @Override - public String toString() { - return Utils.toSqlString("PhysicalOneRowRelation[" + id.asInt() + "]" + getGroupIdAsString(), - "expressions", projects - ); - } - @Override public boolean equals(Object o) { if (this == o) { @@ -99,19 +94,29 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } + if (!super.equals(o)) { + return false; + } PhysicalOneRowRelation that = (PhysicalOneRowRelation) o; return Objects.equals(projects, that.projects); } @Override public int hashCode() { - return Objects.hash(projects); + return Objects.hash(super.hashCode(), projects); + } + + @Override + public String toString() { + return Utils.toSqlString("PhysicalOneRowRelation[" + id.asInt() + "]" + getGroupIdAsString(), + "expressions", projects + ); } @Override public PhysicalOneRowRelation withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalOneRowRelation(projects, groupExpression, + return new PhysicalOneRowRelation(relationId, projects, groupExpression, logicalPropertiesSupplier.get(), physicalProperties, statistics); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRelation.java index d5d47756d53974..49b42bb02a344d 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRelation.java @@ -21,9 +21,9 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.PlanType; -import org.apache.doris.nereids.trees.plans.algebra.Scan; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.algebra.Relation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.statistics.Statistics; @@ -36,34 +36,27 @@ /** * Abstract class for all physical scan plan. */ -public abstract class PhysicalRelation extends PhysicalLeaf implements Scan { +public abstract class PhysicalRelation extends PhysicalLeaf implements Relation { - protected final ObjectId id; - protected final ImmutableList qualifier; + protected final RelationId relationId; /** * Constructor for PhysicalRelation. */ - public PhysicalRelation(ObjectId id, PlanType type, List qualifier, + public PhysicalRelation(RelationId relationId, PlanType type, Optional groupExpression, LogicalProperties logicalProperties) { super(type, groupExpression, logicalProperties); - this.id = id; - this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier, "qualifier can not be null")); + this.relationId = relationId; } /** * Constructor for PhysicalRelation. */ - public PhysicalRelation(ObjectId id, PlanType type, List qualifier, + public PhysicalRelation(RelationId relationId, PlanType type, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { super(type, groupExpression, logicalProperties, physicalProperties, statistics); - this.id = id; - this.qualifier = ImmutableList.copyOf(Objects.requireNonNull(qualifier, "qualifier can not be null")); - } - - public List getQualifier() { - return qualifier; + this.relationId = relationId; } @Override @@ -75,12 +68,12 @@ public boolean equals(Object o) { return false; } PhysicalRelation that = (PhysicalRelation) o; - return this.id.equals(that.id) && Objects.equals(qualifier, that.qualifier); + return Objects.equals(relationId, that.relationId); } @Override public int hashCode() { - return Objects.hash(qualifier); + return Objects.hash(relationId); } @Override @@ -93,7 +86,7 @@ public List getExpressions() { return ImmutableList.of(); } - public ObjectId getId() { - return id; + public RelationId getRelationId() { + return relationId; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSchemaScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSchemaScan.java index 44e85777c0aaf7..ea7c008c801094 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSchemaScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSchemaScan.java @@ -17,14 +17,13 @@ package org.apache.doris.nereids.trees.plans.physical; -import org.apache.doris.catalog.Table; +import org.apache.doris.catalog.SchemaTable; 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.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; -import org.apache.doris.nereids.trees.plans.algebra.Scan; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; @@ -35,27 +34,23 @@ /** * PhysicalSchemaScan. */ -public class PhysicalSchemaScan extends PhysicalRelation implements Scan { +public class PhysicalSchemaScan extends PhysicalCatalogRelation { - private final Table table; - - public PhysicalSchemaScan(ObjectId id, Table table, List qualifier, + public PhysicalSchemaScan(RelationId id, SchemaTable table, List qualifier, Optional groupExpression, LogicalProperties logicalProperties) { - super(id, PlanType.PHYSICAL_SCHEMA_SCAN, qualifier, groupExpression, logicalProperties); - this.table = table; + super(id, PlanType.PHYSICAL_SCHEMA_SCAN, table, qualifier, groupExpression, logicalProperties); } - public PhysicalSchemaScan(ObjectId id, Table table, List qualifier, + public PhysicalSchemaScan(RelationId id, SchemaTable table, List qualifier, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { - super(id, PlanType.PHYSICAL_SCHEMA_SCAN, qualifier, groupExpression, logicalProperties, physicalProperties, - statistics); - this.table = table; + super(id, PlanType.PHYSICAL_SCHEMA_SCAN, table, qualifier, groupExpression, + logicalProperties, physicalProperties, statistics); } @Override - public Table getTable() { - return table; + public SchemaTable getTable() { + return (SchemaTable) table; } @Override @@ -65,32 +60,22 @@ public R accept(PlanVisitor visitor, C context) { @Override public Plan withGroupExpression(Optional groupExpression) { - return new PhysicalSchemaScan(id, table, qualifier, groupExpression, getLogicalProperties(), physicalProperties, - statistics); + return new PhysicalSchemaScan(relationId, getTable(), qualifier, + groupExpression, getLogicalProperties(), physicalProperties, statistics); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalSchemaScan(id, table, qualifier, groupExpression, logicalProperties.get(), - physicalProperties, statistics); + return new PhysicalSchemaScan(relationId, getTable(), qualifier, + groupExpression, logicalProperties.get(), physicalProperties, statistics); } @Override public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalSchemaScan(id, table, qualifier, groupExpression, getLogicalProperties(), physicalProperties, - statistics); - } - - @Override - public boolean equals(Object o) { - return super.equals(o); - } - - @Override - public int hashCode() { - return super.hashCode(); + return new PhysicalSchemaScan(relationId, getTable(), qualifier, + groupExpression, getLogicalProperties(), physicalProperties, statistics); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalStorageLayerAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalStorageLayerAggregate.java index 094f5d75cd5dc5..7a9550adc321f8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalStorageLayerAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalStorageLayerAggregate.java @@ -17,7 +17,6 @@ package org.apache.doris.nereids.trees.plans.physical; -import org.apache.doris.catalog.Table; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; @@ -38,23 +37,23 @@ import java.util.Optional; /** PhysicalStorageLayerAggregate */ -public class PhysicalStorageLayerAggregate extends PhysicalRelation { +public class PhysicalStorageLayerAggregate extends PhysicalCatalogRelation { - private final PhysicalRelation relation; + private final PhysicalCatalogRelation relation; private final PushDownAggOp aggOp; - public PhysicalStorageLayerAggregate(PhysicalRelation relation, PushDownAggOp aggOp) { - super(relation.getId(), relation.getType(), relation.getQualifier(), + public PhysicalStorageLayerAggregate(PhysicalCatalogRelation relation, PushDownAggOp aggOp) { + super(relation.getRelationId(), relation.getType(), relation.getTable(), relation.getQualifier(), Optional.empty(), relation.getLogicalProperties()); this.relation = Objects.requireNonNull(relation, "relation cannot be null"); this.aggOp = Objects.requireNonNull(aggOp, "aggOp cannot be null"); } - public PhysicalStorageLayerAggregate(PhysicalRelation relation, PushDownAggOp aggOp, + public PhysicalStorageLayerAggregate(PhysicalCatalogRelation relation, PushDownAggOp aggOp, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { - super(relation.getId(), relation.getType(), relation.getQualifier(), groupExpression, - logicalProperties, physicalProperties, statistics); + super(relation.getRelationId(), relation.getType(), relation.getTable(), relation.getQualifier(), + groupExpression, logicalProperties, physicalProperties, statistics); this.relation = Objects.requireNonNull(relation, "relation cannot be null"); this.aggOp = Objects.requireNonNull(aggOp, "aggOp cannot be null"); } @@ -67,39 +66,14 @@ public PushDownAggOp getAggOp() { return aggOp; } - @Override - public Table getTable() { - return (Table) relation.getTable(); - } - @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitPhysicalStorageLayerAggregate(this, context); } - @Override - public boolean equals(Object o) { - if (this == o) { - return true; - } - if (o == null || getClass() != o.getClass()) { - return false; - } - if (!super.equals(o)) { - return false; - } - PhysicalStorageLayerAggregate that = (PhysicalStorageLayerAggregate) o; - return Objects.equals(relation, that.relation) && aggOp == that.aggOp; - } - - @Override - public int hashCode() { - return Objects.hash(super.hashCode(), relation, aggOp); - } - @Override public String toString() { - return Utils.toSqlString("PhysicalStorageLayerAggregate[" + id.asInt() + "]" + getGroupIdAsString(), + return Utils.toSqlString("PhysicalStorageLayerAggregate[" + relationId.asInt() + "]" + getGroupIdAsString(), "pushDownAggOp", aggOp, "relation", relation, "stats", statistics @@ -112,8 +86,8 @@ public PhysicalStorageLayerAggregate withPhysicalOlapScan(PhysicalOlapScan physi @Override public PhysicalStorageLayerAggregate withGroupExpression(Optional groupExpression) { - return new PhysicalStorageLayerAggregate(relation, aggOp, groupExpression, getLogicalProperties(), - physicalProperties, statistics); + return new PhysicalStorageLayerAggregate(relation, aggOp, groupExpression, + getLogicalProperties(), physicalProperties, statistics); } @Override @@ -127,7 +101,7 @@ public Plan withGroupExprLogicalPropChildren(Optional groupExpr public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { return new PhysicalStorageLayerAggregate( - (PhysicalRelation) relation.withPhysicalPropertiesAndStats(null, statistics), + (PhysicalCatalogRelation) relation.withPhysicalPropertiesAndStats(null, statistics), aggOp, groupExpression, getLogicalProperties(), physicalProperties, statistics); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTVFRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTVFRelation.java index dfe5ad11689d5d..955ea2f45da522 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTVFRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalTVFRelation.java @@ -17,14 +17,15 @@ package org.apache.doris.nereids.trees.plans.physical; -import org.apache.doris.catalog.FunctionGenTable; 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.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.functions.table.TableValuedFunction; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.algebra.TVFRelation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -41,54 +42,75 @@ public class PhysicalTVFRelation extends PhysicalRelation implements TVFRelation private final TableValuedFunction function; - public PhysicalTVFRelation(ObjectId id, TableValuedFunction function, LogicalProperties logicalProperties) { - super(id, PlanType.PHYSICAL_TVF_RELATION, - ImmutableList.of(), Optional.empty(), logicalProperties); + public PhysicalTVFRelation(RelationId id, TableValuedFunction function, LogicalProperties logicalProperties) { + super(id, PlanType.PHYSICAL_TVF_RELATION, Optional.empty(), logicalProperties); this.function = Objects.requireNonNull(function, "function can not be null"); } - public PhysicalTVFRelation(ObjectId id, TableValuedFunction function, Optional groupExpression, - LogicalProperties logicalProperties, PhysicalProperties physicalProperties, - Statistics statistics) { - super(id, PlanType.PHYSICAL_TVF_RELATION, ImmutableList.of(), groupExpression, logicalProperties, - physicalProperties, statistics); + public PhysicalTVFRelation(RelationId id, TableValuedFunction function, Optional groupExpression, + LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { + super(id, PlanType.PHYSICAL_TVF_RELATION, groupExpression, + logicalProperties, physicalProperties, statistics); this.function = Objects.requireNonNull(function, "function can not be null"); } @Override public PhysicalTVFRelation withGroupExpression(Optional groupExpression) { - return new PhysicalTVFRelation(id, function, groupExpression, getLogicalProperties(), + return new PhysicalTVFRelation(relationId, function, groupExpression, getLogicalProperties(), physicalProperties, statistics); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalTVFRelation(id, function, groupExpression, + return new PhysicalTVFRelation(relationId, function, groupExpression, logicalProperties.get(), physicalProperties, statistics); } @Override public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalTVFRelation(id, function, Optional.empty(), + return new PhysicalTVFRelation(relationId, function, Optional.empty(), getLogicalProperties(), physicalProperties, statistics); } @Override - public FunctionGenTable getTable() { - return function.getTable(); + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + PhysicalTVFRelation that = (PhysicalTVFRelation) o; + return Objects.equals(function, that.function); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), function); } @Override public String toString() { return Utils.toSqlString("PhysicalTVFRelation", - "qualified", Utils.qualifiedName(qualifier, getTable().getName()), + "qualified", Utils.qualifiedName(ImmutableList.of(), function.getTable().getName()), "output", getOutput(), "function", function.toSql() ); } + @Override + public List computeOutput() { + return function.getTable().getBaseSchema() + .stream() + .map(col -> SlotReference.fromColumn(col, ImmutableList.of())) + .collect(ImmutableList.toImmutableList()); + } + @Override public R accept(PlanVisitor visitor, C context) { return visitor.visitPhysicalTVFRelation(this, context); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/RelationUtil.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/RelationUtil.java index 767e7624da4457..dc401a349c26a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/RelationUtil.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/RelationUtil.java @@ -23,12 +23,9 @@ import org.apache.doris.cluster.ClusterNamespace; import org.apache.doris.common.Pair; import org.apache.doris.datasource.CatalogIf; -import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.qe.ConnectContext; -import com.google.common.annotations.VisibleForTesting; import com.google.common.collect.ImmutableList; import java.util.List; @@ -38,28 +35,6 @@ */ public class RelationUtil { - // for test only - private static StatementContext statementContext = new StatementContext(); - - public static ObjectId newRelationId() { - // this branch is for test only - if (ConnectContext.get() == null || ConnectContext.get().getStatementContext() == null) { - return statementContext.getNextObjectId(); - } - return ConnectContext.get().getStatementContext().getNextObjectId(); - } - - /** - * Reset Id Generator - */ - @VisibleForTesting - public static void clear() throws Exception { - if (ConnectContext.get() != null) { - ConnectContext.get().setStatementContext(new StatementContext()); - } - statementContext = new StatementContext(); - } - /** * get table qualifier */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java index 4b4a4eba9d9c03..d10e4a81a6b6a1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java @@ -997,19 +997,13 @@ public int getBeNumberForTest() { needForward = false) public int externalTableAnalyzePartNum = -1; - @VariableMgr.VarAttr( - name = INLINE_CTE_REFERENCED_THRESHOLD - ) + @VariableMgr.VarAttr(name = INLINE_CTE_REFERENCED_THRESHOLD) public int inlineCTEReferencedThreshold = 1; - @VariableMgr.VarAttr( - name = ENABLE_CTE_MATERIALIZE - ) + @VariableMgr.VarAttr(name = ENABLE_CTE_MATERIALIZE) public boolean enableCTEMaterialize = true; - @VariableMgr.VarAttr( - name = IGNORE_COMPLEX_TYPE_COLUMN - ) + @VariableMgr.VarAttr(name = IGNORE_COMPLEX_TYPE_COLUMN) public boolean ignoreColumnWithComplexType = false; @VariableMgr.VarAttr(name = ENABLE_STRONG_CONSISTENCY, description = {"用以开启强一致读。Doris 默认支持同一个会话内的" diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/datasets/tpch/TPCHTestBase.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/datasets/tpch/TPCHTestBase.java index 71dddf165caa95..42dee98ba989d0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/datasets/tpch/TPCHTestBase.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/datasets/tpch/TPCHTestBase.java @@ -20,6 +20,10 @@ public abstract class TPCHTestBase extends AnalyzeCheckTestBase { @Override protected void runBeforeAll() throws Exception { + // The internal table for TPCHTestBase is constructed in order to facilitate + // the execution of certain tests that require the invocation of a deriveStats job. + // This deriveStats job is responsible for retrieving statistics from the aforementioned + // internal table. createDatabase("tpch"); connectContext.setDatabase("default_cluster:tpch"); TPCHUtils.createTables(this); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java index d59e72e860ea57..be5a14343da6c6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslatorTest.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.plans.PreAggStatus; @@ -33,7 +34,6 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.planner.OlapScanNode; import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.PlanNode; @@ -64,7 +64,7 @@ public void testOlapPrune(@Injectable LogicalProperties placeHolder) throws Exce t1Output.add(col2); t1Output.add(col3); LogicalProperties t1Properties = new LogicalProperties(() -> t1Output); - PhysicalOlapScan scan = new PhysicalOlapScan(RelationUtil.newRelationId(), t1, qualifier, t1.getBaseIndexId(), + PhysicalOlapScan scan = new PhysicalOlapScan(StatementScopeIdGenerator.newRelationId(), t1, qualifier, t1.getBaseIndexId(), Collections.emptyList(), Collections.emptyList(), null, PreAggStatus.on(), ImmutableList.of(), Optional.empty(), t1Properties); Literal t1FilterRight = new IntegerLiteral(1); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/RewriteTopDownJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/RewriteTopDownJobTest.java index 8eb03d5b659940..6f1c2d9d9594d4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/RewriteTopDownJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/RewriteTopDownJobTest.java @@ -27,17 +27,17 @@ import org.apache.doris.nereids.rules.rewrite.RewriteRuleFactory; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.logical.LogicalCatalogRelation; 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.LogicalRelation; import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.nereids.types.StringType; import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -70,7 +70,7 @@ public List buildRules() { @Test public void testSimplestScene() { - Plan leaf = new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test")); + Plan leaf = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test")); LogicalProject project = new LogicalProject<>(ImmutableList.of( new SlotReference("name", StringType.INSTANCE, true, ImmutableList.of("test"))), leaf @@ -101,15 +101,15 @@ public void testSimplestScene() { }); } - private static class LogicalBoundRelation extends LogicalRelation { + private static class LogicalBoundRelation extends LogicalCatalogRelation { public LogicalBoundRelation(TableIf table, List qualifier) { - super(RelationUtil.newRelationId(), PlanType.LOGICAL_BOUND_RELATION, table, qualifier); + super(StatementScopeIdGenerator.newRelationId(), PlanType.LOGICAL_BOUND_RELATION, table, qualifier); } public LogicalBoundRelation(TableIf table, List qualifier, Optional groupExpression, Optional logicalProperties) { - super(RelationUtil.newRelationId(), PlanType.LOGICAL_BOUND_RELATION, table, qualifier, + super(StatementScopeIdGenerator.newRelationId(), PlanType.LOGICAL_BOUND_RELATION, table, qualifier, groupExpression, logicalProperties); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJobTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJobTest.java index e4c031e4a8142e..c6c60c815b6a0a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJobTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/jobs/cascades/DeriveStatsJobTest.java @@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; import org.apache.doris.nereids.trees.plans.Plan; @@ -32,7 +33,6 @@ import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.qe.ConnectContext; import org.apache.doris.statistics.Statistics; @@ -81,7 +81,7 @@ private LogicalOlapScan constructOlapSCan() { }}; OlapTable table1 = PlanConstructor.newOlapTable(tableId1, "t1", 0); - return (LogicalOlapScan) new LogicalOlapScan(RelationUtil.newRelationId(), table1, + return (LogicalOlapScan) new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), table1, Collections.emptyList()).withGroupExprLogicalPropChildren(Optional.empty(), Optional.of(new LogicalProperties(() -> ImmutableList.of(slot1))), ImmutableList.of()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java index 43b1c48761baae..3f55ba9579200c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.plans.FakePlan; import org.apache.doris.nereids.trees.plans.GroupPlan; @@ -46,7 +47,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableList; @@ -178,7 +178,7 @@ public void testInsertSameGroup() { @Test public void initByOneLevelPlan() { OlapTable table = PlanConstructor.newOlapTable(0, "a", 1); - LogicalOlapScan scan = new LogicalOlapScan(RelationUtil.newRelationId(), table); + LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), table); PlanChecker.from(connectContext, scan) .checkGroupNum(1) @@ -204,7 +204,7 @@ public void initByTwoLevelChainPlan() { @Test public void initByJoinSameUnboundTable() { - UnboundRelation scanA = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("a")); + UnboundRelation scanA = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("a")); // when unboundRelation contains id, the case is illegal. LogicalJoin topJoin = new LogicalJoin<>(JoinType.INNER_JOIN, scanA, scanA); @@ -215,8 +215,8 @@ public void initByJoinSameUnboundTable() { @Test public void initByJoinSameLogicalTable() { OlapTable tableA = PlanConstructor.newOlapTable(0, "a", 1); - LogicalOlapScan scanA = new LogicalOlapScan(RelationUtil.newRelationId(), tableA); - LogicalOlapScan scanA1 = new LogicalOlapScan(RelationUtil.newRelationId(), tableA); + LogicalOlapScan scanA = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableA); + LogicalOlapScan scanA1 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableA); LogicalJoin topJoin = new LogicalJoin<>(JoinType.INNER_JOIN, scanA, scanA1); @@ -234,8 +234,8 @@ public void initByJoinSameLogicalTable() { public void initByTwoLevelJoinPlan() { OlapTable tableA = PlanConstructor.newOlapTable(0, "a", 1); OlapTable tableB = PlanConstructor.newOlapTable(0, "b", 1); - LogicalOlapScan scanA = new LogicalOlapScan(RelationUtil.newRelationId(), tableA); - LogicalOlapScan scanB = new LogicalOlapScan(RelationUtil.newRelationId(), tableB); + LogicalOlapScan scanA = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableA); + LogicalOlapScan scanB = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableB); LogicalJoin topJoin = new LogicalJoin<>(JoinType.INNER_JOIN, scanA, scanB); @@ -274,10 +274,10 @@ public void initByThreeLevelBushyPlan() { OlapTable tableB = PlanConstructor.newOlapTable(0, "b", 1); OlapTable tableC = PlanConstructor.newOlapTable(0, "c", 1); OlapTable tableD = PlanConstructor.newOlapTable(0, "d", 1); - LogicalOlapScan scanA = new LogicalOlapScan(RelationUtil.newRelationId(), tableA); - LogicalOlapScan scanB = new LogicalOlapScan(RelationUtil.newRelationId(), tableB); - LogicalOlapScan scanC = new LogicalOlapScan(RelationUtil.newRelationId(), tableC); - LogicalOlapScan scanD = new LogicalOlapScan(RelationUtil.newRelationId(), tableD); + LogicalOlapScan scanA = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableA); + LogicalOlapScan scanB = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableB); + LogicalOlapScan scanC = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableC); + LogicalOlapScan scanD = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), tableD); LogicalJoin leftJoin = new LogicalJoin<>(JoinType.CROSS_JOIN, scanA, scanB); LogicalJoin rightJoin = new LogicalJoin<>(JoinType.CROSS_JOIN, scanC, scanD); @@ -307,9 +307,9 @@ public void initByThreeLevelBushyPlan() { */ @Test public void a2a() { - UnboundRelation student = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); PlanChecker.from(connectContext, student) - .applyBottomUp( + .applyBottomUpInMemo( unboundRelation().then(scan -> scan) ) .checkGroupNum(1) @@ -323,10 +323,10 @@ public void a2a() { */ @Test public void a2b() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); - PlanChecker.from(connectContext, new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student"))) - .applyBottomUp( + PlanChecker.from(connectContext, new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student"))) + .applyBottomUpInMemo( unboundRelation().then(scan -> student) ) .checkGroupNum(1) @@ -340,13 +340,13 @@ public void a2b() { */ @Test public void a2newA() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); PlanChecker.from(connectContext, student) - .applyBottomUp( + .applyBottomUpInMemo( logicalOlapScan() .when(scan -> student == scan) - .then(scan -> new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student)) + .then(scan -> new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student)) ) .checkGroupNum(1) .matchesFromRoot(logicalOlapScan().when(student::equals)); @@ -361,11 +361,11 @@ public void a2newA() { */ @Test public void a2bc() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, student); - PlanChecker.from(connectContext, new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student"))) - .applyBottomUp( + PlanChecker.from(connectContext, new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student"))) + .applyBottomUpInMemo( unboundRelation().then(unboundRelation -> limit.withChildren(student)) ) .checkGroupNum(2) @@ -396,11 +396,11 @@ public void a2bc() { public void a2ba() { // invalid case Assertions.assertThrows(IllegalStateException.class, () -> { - UnboundRelation student = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, student) - .applyBottomUp( + .applyBottomUpInMemo( unboundRelation().then(limit::withChildren) ) .checkGroupNum(2) @@ -412,17 +412,17 @@ public void a2ba() { }); // use relation id to divide different unbound relation. - UnboundRelation a = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation a = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); - UnboundRelation a2 = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation a2 = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, a2); PlanChecker.from(connectContext, a) .setMaxInvokeTimesPerRule(1000) - .applyBottomUp( + .applyBottomUpInMemo( unboundRelation() - .when(unboundRelation -> unboundRelation.getId().equals(a.getId())) + .when(unboundRelation -> unboundRelation.getRelationId().equals(a.getRelationId())) .then(unboundRelation -> limit.withChildren( - new UnboundRelation(a2.getId(), unboundRelation.getNameParts())))) + new UnboundRelation(a2.getRelationId(), unboundRelation.getNameParts())))) .checkGroupNum(2) .matchesFromRoot( logicalLimit( @@ -454,9 +454,9 @@ public void a2ba() { /*@Test() public void a2ab() { Assertions.assertThrows(IllegalStateException.class, () -> { - UnboundRelation student = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit = new LogicalLimit<>(1, 0, student); - LogicalOlapScan boundStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan boundStudent = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); CascadesContext cascadesContext = MemoTestUtils.createCascadesContext(connectContext, limit); PlanChecker.from(cascadesContext) @@ -479,12 +479,12 @@ public void a2ab() { */ @Test public void a2bcd() { - LogicalOlapScan scan = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, scan); LogicalLimit> limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, limit5); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( unboundRelation().then(r -> limit10) ) .checkGroupNum(3) @@ -507,11 +507,11 @@ public void a2bcd() { */ @Test public void ab2a() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit().when(limit10::equals).then(limit -> limit) ) .checkGroupNum(2) @@ -531,11 +531,11 @@ public void ab2a() { */ @Test public void ab2NewA() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit().when(limit10::equals).then(limit -> limit.withChildren(limit.child())) ) .checkGroupNum(2) @@ -555,11 +555,11 @@ public void ab2NewA() { */ @Test public void ab2GroupB() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit().when(limit10::equals).then(limit -> limit.child()) ) .checkGroupNum(1) @@ -577,11 +577,11 @@ public void ab2GroupB() { */ @Test public void ab2PlanB() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit(logicalOlapScan()).when(limit10::equals).then(limit -> limit.child()) ) .checkGroupNum(1) @@ -595,16 +595,16 @@ public void ab2PlanB() { * * limit(10) * | -> logicalOlapScan(student) - * UnboundRelation(RelationUtil.newRelationId(), student) + * UnboundRelation(StatementScopeIdGenerator.newRelationId(), student) */ @Test public void ab2c() { - UnboundRelation relation = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation relation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, relation); - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit(unboundRelation()).then(limit -> student) ) .checkGroupNum(1) @@ -622,14 +622,14 @@ public void ab2c() { */ @Test public void ab2cd() { - UnboundRelation relation = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation relation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, relation); - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit(unboundRelation()).then(limit -> limit5) ) .checkGroupNum(2) @@ -650,12 +650,12 @@ public void ab2cd() { */ @Test public void ab2cb() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit().when(limit10::equals).then(limit -> limit5) ) .checkGroupNum(2) @@ -681,14 +681,14 @@ public void ab2cb() { public void ab2NewANewB() { Assertions.assertThrowsExactly(IllegalStateException.class, () -> { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, limit10) .setMaxInvokeTimesPerRule(1000) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit().when(limit10::equals).then(limit -> limit.withChildren( - new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student) + new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student) )) ); }); @@ -706,13 +706,13 @@ public void ab2NewANewB() { @Test public void ab2ba() { Assertions.assertThrowsExactly(IllegalStateException.class, () -> { - UnboundRelation student = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, student); LogicalLimit> limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, limit5); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit(logicalLimit(unboundRelation())).when(limit10::equals).then(l -> l.child().withChildren( l @@ -733,15 +733,15 @@ public void ab2ba() { */ @Test public void ab2cde() { - UnboundRelation student = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit3 = new LogicalLimit<>(3, 0, LimitPhase.ORIGIN, student); - LogicalOlapScan scan = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, scan); LogicalLimit> limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, limit5); PlanChecker.from(connectContext, limit3) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit(unboundRelation()).then(l -> limit10) ) .checkGroupNum(3) @@ -765,13 +765,13 @@ public void ab2cde() { */ @Test public void abc2bac() { - UnboundRelation student = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, student); LogicalLimit> limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, limit5); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit(logicalLimit(unboundRelation())).when(limit10::equals).then(l -> // limit 5 l.child().withChildren( @@ -804,13 +804,13 @@ public void abc2bac() { */ @Test public void abc2bc() { - UnboundRelation student = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("student")); + UnboundRelation student = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("student")); LogicalLimit limit5 = new LogicalLimit<>(5, 0, LimitPhase.ORIGIN, student); LogicalLimit> limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, limit5); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalLimit(logicalLimit(unboundRelation())).then(l -> // limit 10 l.withChildren( @@ -829,13 +829,13 @@ public void abc2bc() { @Test public void testRewriteBottomPlanToOnePlan() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, student); - LogicalOlapScan score = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + LogicalOlapScan score = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); PlanChecker.from(connectContext, limit) - .applyBottomUp( + .applyBottomUpInMemo( logicalOlapScan().when(scan -> Objects.equals(student, scan)).then(scan -> score) ) .checkGroupNum(2) @@ -848,14 +848,14 @@ public void testRewriteBottomPlanToOnePlan() { @Test public void testRewriteBottomPlanToMultiPlan() { - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); - LogicalOlapScan score = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + LogicalOlapScan score = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit1 = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, score); PlanChecker.from(connectContext, limit10) - .applyBottomUp( + .applyBottomUpInMemo( logicalOlapScan().when(scan -> Objects.equals(student, scan)).then(scan -> limit1) ) .checkGroupNum(3) @@ -870,15 +870,15 @@ public void testRewriteBottomPlanToMultiPlan() { @Test public void testRewriteUnboundPlanToBound() { - UnboundRelation unboundTable = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("score")); - LogicalOlapScan boundTable = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + UnboundRelation unboundTable = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("score")); + LogicalOlapScan boundTable = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); PlanChecker.from(connectContext, unboundTable) .checkMemo(memo -> { LogicalProperties logicalProperties = memo.getRoot().getLogicalProperties(); Assertions.assertTrue(logicalProperties instanceof UnboundLogicalProperties); }) - .applyBottomUp(unboundRelation().then(unboundRelation -> boundTable)) + .applyBottomUpInMemo(unboundRelation().then(unboundRelation -> boundTable)) .checkGroupNum(1) .checkMemo(memo -> { LogicalProperties logicalProperties = memo.getRoot().getLogicalProperties(); @@ -892,10 +892,10 @@ public void testRewriteUnboundPlanToBound() { @Test public void testRecomputeLogicalProperties() { - UnboundRelation unboundTable = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("score")); + UnboundRelation unboundTable = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("score")); LogicalLimit unboundLimit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, unboundTable); - LogicalOlapScan boundTable = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + LogicalOlapScan boundTable = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit boundLimit = unboundLimit.withChildren(ImmutableList.of(boundTable)); PlanChecker.from(connectContext, unboundLimit) @@ -903,8 +903,8 @@ public void testRecomputeLogicalProperties() { LogicalProperties logicalProperties = memo.getRoot().getLogicalProperties(); Assertions.assertTrue(logicalProperties instanceof UnboundLogicalProperties); }) - .applyBottomUp(unboundRelation().then(unboundRelation -> boundTable)) - .applyBottomUp( + .applyBottomUpInMemo(unboundRelation().then(unboundRelation -> boundTable)) + .applyBottomUpInMemo( logicalPlan() .when(plan -> plan.canBind() && !(plan instanceof LeafPlan)) .then(LogicalPlan::recomputeLogicalProperties) @@ -924,11 +924,11 @@ public void testRecomputeLogicalProperties() { @Test public void testEliminateRootWithChildGroupInTwoLevels() { - LogicalOlapScan scan = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, scan); PlanChecker.from(connectContext, limit) - .applyBottomUp(logicalLimit().then(LogicalLimit::child)) + .applyBottomUpInMemo(logicalLimit().then(LogicalLimit::child)) .checkGroupNum(1) .checkGroupExpressionNum(1) .checkFirstRootLogicalPlan(scan); @@ -936,11 +936,11 @@ public void testEliminateRootWithChildGroupInTwoLevels() { @Test public void testEliminateRootWithChildPlanInTwoLevels() { - LogicalOlapScan scan = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, scan); PlanChecker.from(connectContext, limit) - .applyBottomUp(logicalLimit(any()).then(LogicalLimit::child)) + .applyBottomUpInMemo(logicalLimit(any()).then(LogicalLimit::child)) .checkGroupNum(1) .checkGroupExpressionNum(1) .checkFirstRootLogicalPlan(scan); @@ -948,19 +948,19 @@ public void testEliminateRootWithChildPlanInTwoLevels() { @Test public void testEliminateTwoLevelsToOnePlan() { - LogicalOlapScan score = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + LogicalOlapScan score = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, score); - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); PlanChecker.from(connectContext, limit) - .applyBottomUp(logicalLimit(any()).then(l -> student)) + .applyBottomUpInMemo(logicalLimit(any()).then(l -> student)) .checkGroupNum(1) .checkGroupExpressionNum(1) .checkFirstRootLogicalPlan(student); PlanChecker.from(connectContext, limit) - .applyBottomUp(logicalLimit(group()).then(l -> student)) + .applyBottomUpInMemo(logicalLimit(group()).then(l -> student)) .checkGroupNum(1) .checkGroupExpressionNum(1) .checkFirstRootLogicalPlan(student); @@ -968,14 +968,14 @@ public void testEliminateTwoLevelsToOnePlan() { @Test public void testEliminateTwoLevelsToTwoPlans() { - LogicalOlapScan score = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + LogicalOlapScan score = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); LogicalLimit limit1 = new LogicalLimit<>(1, 0, LimitPhase.ORIGIN, score); - LogicalOlapScan student = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan student = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalLimit limit10 = new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, student); PlanChecker.from(connectContext, limit1) - .applyBottomUp(logicalLimit(any()).when(limit1::equals).then(l -> limit10)) + .applyBottomUpInMemo(logicalLimit(any()).when(limit1::equals).then(l -> limit10)) .checkGroupNum(2) .checkGroupExpressionNum(2) .matchesFromRoot( @@ -985,7 +985,7 @@ public void testEliminateTwoLevelsToTwoPlans() { ); PlanChecker.from(connectContext, limit1) - .applyBottomUp(logicalLimit(group()).when(limit1::equals).then(l -> limit10)) + .applyBottomUpInMemo(logicalLimit(group()).when(limit1::equals).then(l -> limit10)) .checkGroupNum(2) .checkGroupExpressionNum(2) .matchesFromRoot( @@ -1001,11 +1001,11 @@ public void test() { .analyze(new LogicalLimit<>(10, 0, LimitPhase.ORIGIN, new LogicalJoin<>(JoinType.LEFT_OUTER_JOIN, ImmutableList.of(new EqualTo(new UnboundSlot("sid"), new UnboundSlot("id"))), - new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score), - new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student) + new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score), + new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student) ) )) - .applyTopDown( + .applyTopDownInMemo( logicalLimit(logicalJoin()).then(limit -> { LogicalJoin join = limit.child(); switch (join.getJoinType()) { @@ -1056,7 +1056,7 @@ public void test() { */ @Test public void testRewriteMiddlePlans() { - UnboundRelation unboundRelation = new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test")); + UnboundRelation unboundRelation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test")); LogicalProject insideProject = new LogicalProject<>( ImmutableList.of(new SlotReference("name", StringType.INSTANCE, true, ImmutableList.of("test"))), unboundRelation @@ -1113,7 +1113,7 @@ public void testRewriteMiddlePlans() { */ @Test public void testEliminateRootWithChildPlanThreeLevels() { - UnboundRelation unboundRelation = new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test")); + UnboundRelation unboundRelation = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test")); LogicalProject insideProject = new LogicalProject<>( ImmutableList.of(new SlotReference("inside", StringType.INSTANCE, true, ImmutableList.of("test"))), unboundRelation diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/pattern/GroupExpressionMatchingTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/pattern/GroupExpressionMatchingTest.java index 47454b8a7b9d36..6c110d0b158819 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/pattern/GroupExpressionMatchingTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/pattern/GroupExpressionMatchingTest.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.rules.RulePromise; import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.JoinType; import org.apache.doris.nereids.trees.plans.Plan; @@ -32,7 +33,6 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.types.StringType; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -48,7 +48,7 @@ public class GroupExpressionMatchingTest { public void testLeafNode() { Pattern pattern = new Pattern<>(PlanType.LOGICAL_UNBOUND_RELATION); - Memo memo = new Memo(new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test"))); + Memo memo = new Memo(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test"))); GroupExpressionMatching groupExpressionMatching = new GroupExpressionMatching(pattern, memo.getRoot().getLogicalExpression()); @@ -65,13 +65,13 @@ public void testDepth2() { Pattern pattern = new Pattern<>(PlanType.LOGICAL_PROJECT, new Pattern<>(PlanType.LOGICAL_UNBOUND_RELATION)); - Plan leaf = new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test")); + Plan leaf = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test")); LogicalProject root = new LogicalProject(ImmutableList .of(new SlotReference("name", StringType.INSTANCE, true, ImmutableList.of("test"))), leaf); Memo memo = new Memo(root); - Plan anotherLeaf = new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test2")); + Plan anotherLeaf = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test2")); memo.copyIn(anotherLeaf, memo.getRoot().getLogicalExpression().child(0), false); GroupExpressionMatching groupExpressionMatching @@ -96,13 +96,13 @@ public void testDepth2() { public void testDepth2WithGroup() { Pattern pattern = new Pattern<>(PlanType.LOGICAL_PROJECT, Pattern.GROUP); - Plan leaf = new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test")); + Plan leaf = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test")); LogicalProject root = new LogicalProject(ImmutableList .of(new SlotReference("name", StringType.INSTANCE, true, ImmutableList.of("test"))), leaf); Memo memo = new Memo(root); - Plan anotherLeaf = new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test2")); + Plan anotherLeaf = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test2")); memo.copyIn(anotherLeaf, memo.getRoot().getLogicalExpression().child(0), false); GroupExpressionMatching groupExpressionMatching @@ -122,7 +122,7 @@ public void testDepth2WithGroup() { public void testLeafAny() { Pattern pattern = Pattern.ANY; - Memo memo = new Memo(new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test"))); + Memo memo = new Memo(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test"))); GroupExpressionMatching groupExpressionMatching = new GroupExpressionMatching(pattern, memo.getRoot().getLogicalExpression()); @@ -139,10 +139,10 @@ public void testAnyWithChild() { Plan root = new LogicalProject( ImmutableList.of(new SlotReference("name", StringType.INSTANCE, true, ImmutableList.of("test"))), - new UnboundRelation(RelationUtil.newRelationId(), Lists.newArrayList("test"))); + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), Lists.newArrayList("test"))); Memo memo = new Memo(root); - Plan anotherLeaf = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("test2")); + Plan anotherLeaf = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("test2")); memo.copyIn(anotherLeaf, memo.getRoot().getLogicalExpression().child(0), false); GroupExpressionMatching groupExpressionMatching @@ -161,8 +161,8 @@ public void testAnyWithChild() { @Test public void testInnerLogicalJoinMatch() { Plan root = new LogicalJoin(JoinType.INNER_JOIN, - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("a")), - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("b")) + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("a")), + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("b")) ); Memo memo = new Memo(root); @@ -183,8 +183,8 @@ public void testInnerLogicalJoinMatch() { @Test public void testInnerLogicalJoinMismatch() { Plan root = new LogicalJoin(JoinType.LEFT_OUTER_JOIN, - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("a")), - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("b")) + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("a")), + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("b")) ); Memo memo = new Memo(root); @@ -200,8 +200,8 @@ public void testInnerLogicalJoinMismatch() { @Test public void testTopMatchButChildrenNotMatch() { Plan root = new LogicalJoin(JoinType.LEFT_OUTER_JOIN, - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("a")), - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("b")) + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("a")), + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("b")) ); Memo memo = new Memo(root); @@ -222,9 +222,9 @@ public void testSubTreeMatch() { new UnboundSlot(Lists.newArrayList("b", "id")))), new LogicalJoin(JoinType.INNER_JOIN, new LogicalJoin(JoinType.LEFT_OUTER_JOIN, - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("a")), - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("b"))), - new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("c"))) + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("a")), + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("b"))), + new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("c"))) ); Pattern p1 = patterns().logicalFilter(patterns().subTree(LogicalFilter.class, LogicalJoin.class)).pattern; Iterator matchResult1 = match(root, p1); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/MergeProjectPostProcessTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/MergeProjectPostProcessTest.java index 43b9c8bae178d7..74822d8fe59366 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/MergeProjectPostProcessTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/MergeProjectPostProcessTest.java @@ -26,8 +26,8 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.PreAggStatus; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; @@ -75,7 +75,7 @@ public void testMergeProj(@Injectable LogicalProperties placeHolder, @Injectable t1Output.add(b); t1Output.add(c); LogicalProperties t1Properties = new LogicalProperties(() -> t1Output); - PhysicalOlapScan scan = new PhysicalOlapScan(ObjectId.createGenerator().getNextId(), t1, qualifier, 0L, + PhysicalOlapScan scan = new PhysicalOlapScan(RelationId.createGenerator().getNextId(), t1, qualifier, 0L, Collections.emptyList(), Collections.emptyList(), null, PreAggStatus.on(), ImmutableList.of(), Optional.empty(), t1Properties); Alias x = new Alias(a, "x"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/PushdownFilterThroughProjectTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/PushdownFilterThroughProjectTest.java index 8c0e701e99a7c4..ccbed847c32848 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/PushdownFilterThroughProjectTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/postprocess/PushdownFilterThroughProjectTest.java @@ -29,8 +29,8 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.PreAggStatus; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; @@ -86,7 +86,7 @@ public void testPushFilter(@Injectable LogicalProperties placeHolder, t1Output.add(b); t1Output.add(c); LogicalProperties t1Properties = new LogicalProperties(() -> t1Output); - PhysicalOlapScan scan = new PhysicalOlapScan(ObjectId.createGenerator().getNextId(), t1, + PhysicalOlapScan scan = new PhysicalOlapScan(RelationId.createGenerator().getNextId(), t1, qualifier, 0L, Collections.emptyList(), Collections.emptyList(), null, PreAggStatus.on(), ImmutableList.of(), Optional.empty(), t1Properties); Alias x = new Alias(a, "x"); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/RegisterCTETest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java similarity index 57% rename from fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/RegisterCTETest.java rename to fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java index cccc56da9e9ae9..7e95409118c256 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/RegisterCTETest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java @@ -18,7 +18,6 @@ package org.apache.doris.nereids.rules.analysis; import org.apache.doris.common.NereidsException; -import org.apache.doris.nereids.CTEContext; import org.apache.doris.nereids.NereidsPlanner; import org.apache.doris.nereids.StatementContext; import org.apache.doris.nereids.datasets.ssb.SSBUtils; @@ -49,35 +48,45 @@ import java.util.List; -public class RegisterCTETest extends TestWithFeService implements MemoPatternMatchSupported { +public class AnalyzeCTETest extends TestWithFeService implements MemoPatternMatchSupported { private final NereidsParser parser = new NereidsParser(); - private final String sql1 = "WITH cte1 AS (SELECT s_suppkey FROM supplier WHERE s_suppkey < 5), " + private final String multiCte = "WITH cte1 AS (SELECT s_suppkey FROM supplier WHERE s_suppkey < 5), " + "cte2 AS (SELECT s_suppkey FROM cte1 WHERE s_suppkey < 3)" + "SELECT * FROM cte1, cte2"; - private final String sql2 = "WITH cte1 (skey) AS (SELECT s_suppkey, s_nation FROM supplier WHERE s_suppkey < 5), " + private final String cteWithColumnAlias = "WITH cte1 (skey) AS (SELECT s_suppkey, s_nation FROM supplier WHERE s_suppkey < 5), " + "cte2 (sk2) AS (SELECT skey FROM cte1 WHERE skey < 3)" + "SELECT * FROM cte1, cte2"; - private final String sql3 = "WITH cte1 AS (SELECT * FROM supplier), " + private final String cteConsumerInSubQuery = "WITH cte1 AS (SELECT * FROM supplier), " + "cte2 AS (SELECT * FROM supplier WHERE s_region in (\"ASIA\", \"AFRICA\"))" + "SELECT s_region, count(*) FROM cte1 GROUP BY s_region HAVING s_region in (SELECT s_region FROM cte2)"; - private final String sql4 = "WITH cte1 AS (SELECT s_suppkey AS sk FROM supplier WHERE s_suppkey < 5), " + private final String cteConsumerJoin = "WITH cte1 AS (SELECT s_suppkey AS sk FROM supplier WHERE s_suppkey < 5), " + "cte2 AS (SELECT sk FROM cte1 WHERE sk < 3)" + "SELECT * FROM cte1 JOIN cte2 ON cte1.sk = cte2.sk"; - private final String sql5 = "WITH V1 AS (SELECT s_suppkey FROM supplier), " + private final String cteReferToAnotherOne = "WITH V1 AS (SELECT s_suppkey FROM supplier), " + "V2 AS (SELECT s_suppkey FROM V1)" + "SELECT * FROM V2"; - private final String sql6 = "WITH cte1 AS (SELECT s_suppkey FROM supplier)" + private final String cteJoinSelf = "WITH cte1 AS (SELECT s_suppkey FROM supplier)" + "SELECT * FROM cte1 AS t1, cte1 AS t2"; - private final List testSql = ImmutableList.of( - sql1, sql2, sql3, sql4, sql5, sql6 + private final String cteNested = "WITH cte1 AS (" + + "WITH cte2 AS (SELECT s_suppkey FROM supplier) SELECT * FROM cte2)" + + " SELECT * FROM cte1"; + + private final String cteInTheMiddle = "SELECT * FROM (WITH cte1 AS (SELECT s_suppkey FROM supplier)" + + " SELECT * FROM cte1) a"; + + private final String cteWithDiffRelationId = "with s as (select * from supplier) select * from s as s1, s as s2"; + + private final List testSqls = ImmutableList.of( + multiCte, cteWithColumnAlias, cteConsumerInSubQuery, cteConsumerJoin, cteReferToAnotherOne, cteJoinSelf, + cteNested, cteInTheMiddle, cteWithDiffRelationId ); @Override @@ -94,13 +103,6 @@ protected void runBeforeEach() throws Exception { StatementScopeIdGenerator.clear(); } - private CTEContext getCTEContextAfterRegisterCTE(String sql) { - return PlanChecker.from(connectContext) - .analyze(sql) - .getCascadesContext() - .getCteContext(); - } - /* ******************************************************************************************** * Test CTE * ******************************************************************************************** */ @@ -114,7 +116,7 @@ public List getExplorationRules() { } }; - for (String sql : testSql) { + for (String sql : testSqls) { StatementScopeIdGenerator.clear(); StatementContext statementContext = MemoTestUtils.createStatementContext(connectContext, sql); PhysicalPlan plan = new NereidsPlanner(statementContext).plan( @@ -126,97 +128,80 @@ public List getExplorationRules() { } } - @Test - public void testCTERegister() { - CTEContext cteContext = getCTEContextAfterRegisterCTE(sql1); - - Assertions.assertTrue(cteContext.containsCTE("cte1") - && cteContext.containsCTE("cte2")); - // LogicalPlan cte2parsedPlan = cteContext.getParsedCtePlan("cte2").get(); - // PlanChecker.from(connectContext, cte2parsedPlan) - // .matchesFromRoot( - // logicalSubQueryAlias( - // logicalProject( - // logicalFilter( - // logicalCheckPolicy( - // unboundRelation() - // ) - // ) - // ) - // ) - // ); - } - - @Test - public void testCTERegisterWithColumnAlias() { - CTEContext cteContext = getCTEContextAfterRegisterCTE(sql2); - - Assertions.assertTrue(cteContext.containsCTE("cte1") - && cteContext.containsCTE("cte2")); - - // check analyzed plan - // LogicalPlan cte1AnalyzedPlan = cteContext.getReuse("cte1").get(); - - // PlanChecker.from(connectContext, cte1AnalyzedPlan) - // .matchesFromRoot( - // logicalSubQueryAlias( - // logicalProject() - // .when(p -> p.getProjects().size() == 2 - // && p.getProjects().get(0).getName().equals("s_suppkey") - // && p.getProjects().get(0).getExprId().asInt() == 14 - // && p.getProjects().get(0).getQualifier().equals(ImmutableList.of("default_cluster:test", "supplier")) - // && p.getProjects().get(1).getName().equals("s_nation") - // && p.getProjects().get(1).getExprId().asInt() == 18 - // && p.getProjects().get(1).getQualifier().equals(ImmutableList.of("default_cluster:test", "supplier")) - // ) - // ) - // .when(a -> a.getAlias().equals("cte1")) - // .when(a -> a.getOutput().size() == 2 - // && a.getOutput().get(0).getName().equals("skey") - // && a.getOutput().get(0).getExprId().asInt() == 14 - // && a.getOutput().get(0).getQualifier().equals(ImmutableList.of("cte1")) - // && a.getOutput().get(1).getName().equals("s_nation") - // && a.getOutput().get(1).getExprId().asInt() == 18 - // && a.getOutput().get(1).getQualifier().equals(ImmutableList.of("cte1")) - // ) - // ); - } - @Test public void testCTEInHavingAndSubquery() { PlanChecker.from(connectContext) - .analyze(sql3) + .analyze(cteConsumerInSubQuery) .applyBottomUp(new PullUpProjectUnderApply()) .applyBottomUp(new UnCorrelatedApplyFilter()) .applyBottomUp(new InApplyToJoin()) .matches( - logicalCTE( - logicalFilter( - logicalProject( - logicalJoin( - logicalAggregate( - logicalCTEConsumer() - ), logicalProject( - logicalCTEConsumer()) - ) - ) - - ) - ) + logicalFilter( + logicalProject( + logicalJoin( + logicalAggregate(), + logicalProject() + ) + ) + + ) ); } @Test public void testCTEWithAlias() { PlanChecker.from(connectContext) - .analyze(sql4) + .analyze(cteConsumerJoin) + .matchesFromRoot( + logicalCTEAnchor( + logicalCTEProducer(), + logicalCTEAnchor( + logicalCTEProducer(), + logicalProject( + logicalJoin( + logicalCTEConsumer(), + logicalCTEConsumer() + ) + ) + ) + ) + ); + } + + @Test + public void testCTEWithAnExistedTableOrViewName() { + PlanChecker.from(connectContext) + .analyze(cteReferToAnotherOne) + .matchesFromRoot( + logicalCTEAnchor( + logicalCTEProducer(), + logicalCTEAnchor( + logicalCTEProducer(), + logicalProject( + logicalCTEConsumer() + ) + ) + ) + ); + + } + + @Test + public void testDifferenceRelationId() { + PlanChecker.from(connectContext) + .analyze(cteWithDiffRelationId) .matchesFromRoot( - logicalCTE( + logicalCTEAnchor( + logicalCTEProducer(), logicalProject( logicalJoin( - logicalCTEConsumer(), - logicalCTEConsumer() + logicalSubQueryAlias( + logicalCTEConsumer() + ), + logicalSubQueryAlias( + logicalCTEConsumer() + ) ) ) ) @@ -224,17 +209,45 @@ public void testCTEWithAlias() { } @Test - public void testCTEWithAnExistedTableOrViewName() { + public void testCteInTheMiddle() { PlanChecker.from(connectContext) - .analyze(sql5) + .analyze(cteInTheMiddle) .matchesFromRoot( - logicalCTE( + logicalProject( + logicalSubQueryAlias( + logicalCTEAnchor( + logicalCTEProducer(), + logicalProject( + logicalCTEConsumer() + ) + ) + ) + ) + + ); + } + + @Test + public void testCteNested() { + PlanChecker.from(connectContext) + .analyze(cteNested) + .matchesFromRoot( + logicalCTEAnchor( + logicalCTEProducer( + logicalSubQueryAlias( + logicalCTEAnchor( + logicalCTEProducer(), + logicalProject( + logicalCTEConsumer() + ) + ) + ) + ), logicalProject( logicalCTEConsumer() ) ) ); - } @@ -247,9 +260,8 @@ public void testCTEExceptionOfDuplicatedColumnAlias() { String sql = "WITH cte1 (a1, A1) AS (SELECT * FROM supplier)" + "SELECT * FROM cte1"; - NereidsException exception = Assertions.assertThrows(NereidsException.class, () -> { - PlanChecker.from(connectContext).checkPlannerResult(sql); - }, "Not throw expected exception."); + NereidsException exception = Assertions.assertThrows(NereidsException.class, + () -> PlanChecker.from(connectContext).checkPlannerResult(sql), "Not throw expected exception."); Assertions.assertTrue(exception.getMessage().contains("Duplicated CTE column alias: [a1] in CTE [cte1]")); } @@ -259,9 +271,8 @@ public void testCTEExceptionOfColumnAliasSize() { + "(SELECT s_suppkey FROM supplier)" + "SELECT * FROM cte1"; - NereidsException exception = Assertions.assertThrows(NereidsException.class, () -> { - PlanChecker.from(connectContext).checkPlannerResult(sql); - }, "Not throw expected exception."); + NereidsException exception = Assertions.assertThrows(NereidsException.class, + () -> PlanChecker.from(connectContext).checkPlannerResult(sql), "Not throw expected exception."); System.out.println(exception.getMessage()); Assertions.assertTrue(exception.getMessage().contains("CTE [cte1] returns 2 columns, " + "but 1 labels were specified.")); @@ -273,9 +284,8 @@ public void testCTEExceptionOfReferenceInWrongOrder() { + "cte2 AS (SELECT * FROM supplier)" + "SELECT * FROM cte1, cte2"; - RuntimeException exception = Assertions.assertThrows(RuntimeException.class, () -> { - PlanChecker.from(connectContext).checkPlannerResult(sql); - }, "Not throw expected exception."); + RuntimeException exception = Assertions.assertThrows(RuntimeException.class, + () -> PlanChecker.from(connectContext).checkPlannerResult(sql), "Not throw expected exception."); Assertions.assertTrue(exception.getMessage().contains("[cte2] does not exist in database")); } @@ -284,9 +294,8 @@ public void testCTEExceptionOfErrorInUnusedCTE() { String sql = "WITH cte1 AS (SELECT * FROM not_existed_table)" + "SELECT * FROM supplier"; - RuntimeException exception = Assertions.assertThrows(RuntimeException.class, () -> { - PlanChecker.from(connectContext).checkPlannerResult(sql); - }, "Not throw expected exception."); + RuntimeException exception = Assertions.assertThrows(RuntimeException.class, + () -> PlanChecker.from(connectContext).checkPlannerResult(sql), "Not throw expected exception."); Assertions.assertTrue(exception.getMessage().contains("[not_existed_table] does not exist in database")); } @@ -296,29 +305,18 @@ public void testCTEExceptionOfDuplicatedCTEName() { + "cte1 AS (SELECT * FROM part)" + "SELECT * FROM cte1"; - AnalysisException exception = Assertions.assertThrows(AnalysisException.class, () -> { - PlanChecker.from(connectContext).analyze(sql); - }, "Not throw expected exception."); + AnalysisException exception = Assertions.assertThrows(AnalysisException.class, + () -> PlanChecker.from(connectContext).analyze(sql), "Not throw expected exception."); Assertions.assertTrue(exception.getMessage().contains("[cte1] cannot be used more than once")); } @Test - public void testDifferenceRelationId() { - PlanChecker.from(connectContext) - .analyze("with s as (select * from supplier) select * from s as s1, s as s2") - .matchesFromRoot( - logicalCTE( - logicalProject( - logicalJoin( - logicalSubQueryAlias( - logicalCTEConsumer() - ), - logicalSubQueryAlias( - logicalCTEConsumer() - ) - ) - ) - ) - ); + public void testCTEExceptionOfRefterCTENameNotInScope() { + String sql = "WITH cte1 AS (WITH cte2 AS (SELECT * FROM supplier) SELECT * FROM cte2)" + + "SELECT * FROM cte2"; + + AnalysisException exception = Assertions.assertThrows(AnalysisException.class, + () -> PlanChecker.from(connectContext).analyze(sql), "Not throw expected exception."); + Assertions.assertTrue(exception.getMessage().contains("Table [cte2] does not exist in database")); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java index f9550dce0b9f30..3a99e91dd84e29 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java @@ -27,11 +27,11 @@ import org.apache.doris.nereids.pattern.GeneratedPlanPatterns; import org.apache.doris.nereids.rules.RulePromise; import org.apache.doris.nereids.rules.analysis.BindRelation.CustomTableResolver; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanRewriter; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.utframe.TestWithFeService; import com.google.common.collect.ImmutableList; @@ -59,7 +59,7 @@ protected void runBeforeAll() throws Exception { @Test void bindInCurrentDb() { connectContext.setDatabase(DEFAULT_CLUSTER_PREFIX + DB1); - Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("t")), + Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("t")), connectContext, new BindRelation()); Assertions.assertTrue(plan instanceof LogicalOlapScan); @@ -71,7 +71,7 @@ void bindInCurrentDb() { @Test void bindByDbQualifier() { connectContext.setDatabase(DEFAULT_CLUSTER_PREFIX + DB2); - Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("db1", "t")), + Plan plan = PlanRewriter.bottomUpRewrite(new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("db1", "t")), connectContext, new BindRelation()); Assertions.assertTrue(plan instanceof LogicalOlapScan); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java index 39dad71e07adfb..0a3334b4cf0c2b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindSlotReferenceTest.java @@ -33,7 +33,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -51,7 +50,7 @@ public void beforeEach() throws Exception { @Test public void testCannotFindSlot() { LogicalProject project = new LogicalProject<>(ImmutableList.of(new UnboundSlot("foo")), - new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student)); + new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student)); AnalysisException exception = Assertions.assertThrows(AnalysisException.class, () -> PlanChecker.from(MemoTestUtils.createConnectContext()).analyze(project)); Assertions.assertEquals("unbounded object foo in PROJECT clause.", exception.getMessage()); @@ -59,8 +58,8 @@ public void testCannotFindSlot() { @Test public void testAmbiguousSlot() { - LogicalOlapScan scan1 = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); - LogicalOlapScan scan2 = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan scan1 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); + LogicalOlapScan scan2 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalJoin join = new LogicalJoin<>( JoinType.CROSS_JOIN, scan1, scan2); LogicalProject> project = new LogicalProject<>( @@ -79,9 +78,9 @@ public void testAmbiguousSlot() { */ @Test public void testGroupByOnJoin() { - LogicalOlapScan scan1 = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan scan1 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalSubQueryAlias sub1 = new LogicalSubQueryAlias("t1", scan1); - LogicalOlapScan scan2 = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan scan2 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalSubQueryAlias sub2 = new LogicalSubQueryAlias("t2", scan2); LogicalJoin, LogicalSubQueryAlias> join = new LogicalJoin<>(JoinType.CROSS_JOIN, sub1, sub2); @@ -105,9 +104,9 @@ public void testGroupByOnJoin() { */ @Test public void testGroupByOnJoinAmbiguous() { - LogicalOlapScan scan1 = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan scan1 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalSubQueryAlias sub1 = new LogicalSubQueryAlias("t1", scan1); - LogicalOlapScan scan2 = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + LogicalOlapScan scan2 = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); LogicalSubQueryAlias sub2 = new LogicalSubQueryAlias("t2", scan2); LogicalJoin, LogicalSubQueryAlias> join = new LogicalJoin<>(JoinType.CROSS_JOIN, sub1, sub2); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckAnalysisTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckAnalysisTest.java index 386d8a17188c55..f470c0aa1f5319 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckAnalysisTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckAnalysisTest.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.And; import org.apache.doris.nereids.trees.expressions.Not; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.plans.GroupPlan; @@ -53,7 +54,7 @@ public void testCheckExpressionInputTypes() { @Test public void testCheckNotWithChildrenWithErrorType() { - Plan plan = new LogicalOneRowRelation( + Plan plan = new LogicalOneRowRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of(new Alias(new Not(new IntegerLiteral(2)), "not_2"))); CheckAnalysis checkAnalysis = new CheckAnalysis(); Assertions.assertThrows(AnalysisException.class, () -> @@ -63,7 +64,7 @@ public void testCheckNotWithChildrenWithErrorType() { @Test public void testUnbound() { UnboundFunction func = new UnboundFunction("now", Lists.newArrayList(new IntegerLiteral(1))); - Plan plan = new LogicalOneRowRelation( + Plan plan = new LogicalOneRowRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of(new Alias(func, "unboundFunction"))); CheckBound checkBound = new CheckBound(); Assertions.assertThrows(AnalysisException.class, () -> diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckRowPolicyTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckRowPolicyTest.java index b7bdad064a3a3f..47e672db99ad78 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckRowPolicyTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckRowPolicyTest.java @@ -34,13 +34,13 @@ import org.apache.doris.common.FeConstants; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalCheckPolicy; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.util.PlanRewriter; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.system.SystemInfoService; import org.apache.doris.thrift.TStorageType; import org.apache.doris.utframe.TestWithFeService; @@ -98,7 +98,7 @@ protected void runBeforeAll() throws Exception { @Test public void checkUser() throws AnalysisException, org.apache.doris.common.AnalysisException { - LogicalRelation relation = new LogicalOlapScan(RelationUtil.newRelationId(), olapTable, Arrays.asList(fullDbName)); + LogicalRelation relation = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), olapTable, Arrays.asList(fullDbName)); LogicalCheckPolicy checkPolicy = new LogicalCheckPolicy<>(relation); useUser("root"); @@ -113,7 +113,7 @@ public void checkUser() throws AnalysisException, org.apache.doris.common.Analys @Test public void checkNoPolicy() throws org.apache.doris.common.AnalysisException { useUser(userName); - LogicalRelation relation = new LogicalOlapScan(RelationUtil.newRelationId(), olapTable, Arrays.asList(fullDbName)); + LogicalRelation relation = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), olapTable, Arrays.asList(fullDbName)); LogicalCheckPolicy checkPolicy = new LogicalCheckPolicy<>(relation); Plan plan = PlanRewriter.bottomUpRewrite(checkPolicy, connectContext, new CheckPolicy()); Assertions.assertEquals(plan, relation); @@ -122,7 +122,7 @@ public void checkNoPolicy() throws org.apache.doris.common.AnalysisException { @Test public void checkOnePolicy() throws Exception { useUser(userName); - LogicalRelation relation = new LogicalOlapScan(RelationUtil.newRelationId(), olapTable, Arrays.asList(fullDbName)); + LogicalRelation relation = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), olapTable, Arrays.asList(fullDbName)); LogicalCheckPolicy checkPolicy = new LogicalCheckPolicy<>(relation); connectContext.getSessionVariable().setEnableNereidsPlanner(true); createPolicy("CREATE ROW POLICY " diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTestHelper.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTestHelper.java index 8ee43dbfd22c5b..9cd4ea49c6463c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTestHelper.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/ExpressionRewriteTestHelper.java @@ -25,7 +25,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.BooleanType; import org.apache.doris.nereids.types.DataType; @@ -52,7 +52,7 @@ public abstract class ExpressionRewriteTestHelper { public ExpressionRewriteTestHelper() { CascadesContext cascadesContext = MemoTestUtils.createCascadesContext( - new UnboundRelation(new ObjectId(1), ImmutableList.of("tbl"))); + new UnboundRelation(new RelationId(1), ImmutableList.of("tbl"))); context = new ExpressionRewriteContext(cascadesContext); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/FoldConstantTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/FoldConstantTest.java index c9f74bc7f2b52e..4f89f77937d9ff 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/FoldConstantTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/FoldConstantTest.java @@ -39,7 +39,7 @@ import org.apache.doris.nereids.trees.expressions.literal.Interval.TimeUnit; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.expressions.literal.VarcharLiteral; -import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.types.DateTimeV2Type; import org.apache.doris.nereids.types.DoubleType; import org.apache.doris.nereids.types.IntegerType; @@ -607,7 +607,7 @@ public void testFoldNestedExpression() { public void testFoldTypeOfNullLiteral() { String actualExpression = "append_trailing_char_if_absent(cast(version() as varchar), cast(null as varchar))"; ExpressionRewriteContext context = new ExpressionRewriteContext( - MemoTestUtils.createCascadesContext(new UnboundRelation(new ObjectId(1), ImmutableList.of("test_table")))); + MemoTestUtils.createCascadesContext(new UnboundRelation(new RelationId(1), ImmutableList.of("test_table")))); NereidsParser parser = new NereidsParser(); Expression e1 = parser.parseExpression(actualExpression); e1 = new ExpressionNormalization().rewrite(FunctionBinder.INSTANCE.rewrite(e1, context), context); @@ -616,7 +616,7 @@ public void testFoldTypeOfNullLiteral() { private void assertRewriteExpression(String actualExpression, String expectedExpression) { ExpressionRewriteContext context = new ExpressionRewriteContext( - MemoTestUtils.createCascadesContext(new UnboundRelation(new ObjectId(1), ImmutableList.of("test_table")))); + MemoTestUtils.createCascadesContext(new UnboundRelation(new RelationId(1), ImmutableList.of("test_table")))); NereidsParser parser = new NereidsParser(); Expression e1 = parser.parseExpression(actualExpression); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java index 68aad77c64464d..8058137b6f65f2 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/expression/SimplifyRangeTest.java @@ -25,7 +25,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.expressions.SlotReference; -import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.types.BooleanType; import org.apache.doris.nereids.types.DataType; @@ -52,7 +52,7 @@ public class SimplifyRangeTest { public SimplifyRangeTest() { CascadesContext cascadesContext = MemoTestUtils.createCascadesContext( - new UnboundRelation(new ObjectId(1), ImmutableList.of("tbl"))); + new UnboundRelation(new RelationId(1), ImmutableList.of("tbl"))); context = new ExpressionRewriteContext(cascadesContext); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindowTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindowTest.java index 277e095b1cac20..c1ac714a93a99e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindowTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/implementation/LogicalWindowToPhysicalWindowTest.java @@ -21,6 +21,7 @@ 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.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.WindowFrame; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; @@ -32,7 +33,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -50,7 +50,7 @@ public class LogicalWindowToPhysicalWindowTest implements MemoPatternMatchSuppor @BeforeAll public final void beforeAll() { - rStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, + rStudent = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student, ImmutableList.of("")); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggScalarSubQueryToWindowFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggScalarSubQueryToWindowFunctionTest.java index 7839681a43f663..f70161f4654958 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggScalarSubQueryToWindowFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggScalarSubQueryToWindowFunctionTest.java @@ -341,7 +341,7 @@ private void check(String sql) { System.out.printf("Test:\n%s\n\n", sql); Plan plan = PlanChecker.from(createCascadesContext(sql)) .analyze(sql) - .applyTopDown(new AggScalarSubQueryToWindowFunction()) + .customRewrite(new AggScalarSubQueryToWindowFunction()) .rewrite() .getPlan(); System.out.println(plan.treeString()); @@ -352,7 +352,7 @@ private void checkNot(String sql) { System.out.printf("Test:\n%s\n\n", sql); Plan plan = PlanChecker.from(createCascadesContext(sql)) .analyze(sql) - .applyTopDown(new AggScalarSubQueryToWindowFunction()) + .customRewrite(new AggScalarSubQueryToWindowFunction()) .rewrite() .getPlan(); System.out.println(plan.treeString()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java index e8419b245850a4..6f3bfaa7e53314 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/AggregateStrategiesTest.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; import org.apache.doris.nereids.trees.expressions.functions.agg.Count; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; @@ -40,7 +41,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -60,7 +60,7 @@ public class AggregateStrategiesTest implements MemoPatternMatchSupported { @BeforeAll public final void beforeAll() { - rStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, + rStudent = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student, ImmutableList.of("")); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/CheckAndStandardizeWindowFunctionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/CheckAndStandardizeWindowFunctionTest.java index 1da3839339d454..ffe88af2fea0ba 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/CheckAndStandardizeWindowFunctionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/CheckAndStandardizeWindowFunctionTest.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.WindowFrame; import org.apache.doris.nereids.trees.expressions.WindowFrame.FrameBoundary; @@ -43,7 +44,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -66,7 +66,7 @@ public class CheckAndStandardizeWindowFunctionTest implements MemoPatternMatchSu @BeforeAll public final void beforeAll() { - rStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, ImmutableList.of()); + rStudent = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student, ImmutableList.of()); gender = rStudent.getOutput().get(1).toSlot(); age = rStudent.getOutput().get(3).toSlot(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstantTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstantTest.java index 8e0642657827c5..2fbf2adf17b1c4 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstantTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateGroupByConstantTest.java @@ -31,7 +31,7 @@ import org.apache.doris.nereids.trees.expressions.functions.agg.Min; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.expressions.literal.StringLiteral; -import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.types.IntegerType; @@ -66,7 +66,7 @@ class EliminateGroupByConstantTest implements MemoPatternMatchSupported { @Test void testIntegerLiteral() { LogicalPlan aggregate = new LogicalPlanBuilder( - new LogicalOlapScan(ObjectId.createGenerator().getNextId(), table)) + new LogicalOlapScan(RelationId.createGenerator().getNextId(), table)) .agg(ImmutableList.of(new IntegerLiteral(1), k2), ImmutableList.of(k1, k2)) .build(); @@ -82,7 +82,7 @@ void testIntegerLiteral() { @Test void testOtherLiteral() { LogicalPlan aggregate = new LogicalPlanBuilder( - new LogicalOlapScan(ObjectId.createGenerator().getNextId(), table)) + new LogicalOlapScan(RelationId.createGenerator().getNextId(), table)) .agg(ImmutableList.of( new StringLiteral("str"), k2), ImmutableList.of( @@ -100,7 +100,7 @@ void testOtherLiteral() { @Test void testMixedLiteral() { LogicalPlan aggregate = new LogicalPlanBuilder( - new LogicalOlapScan(ObjectId.createGenerator().getNextId(), table)) + new LogicalOlapScan(RelationId.createGenerator().getNextId(), table)) .agg(ImmutableList.of( new StringLiteral("str"), k2, new IntegerLiteral(1), @@ -124,7 +124,7 @@ k2, k1, new Alias(new IntegerLiteral(1), "integer"))) @Test void testComplexGroupBy() { LogicalPlan aggregate = new LogicalPlanBuilder( - new LogicalOlapScan(ObjectId.createGenerator().getNextId(), table)) + new LogicalOlapScan(RelationId.createGenerator().getNextId(), table)) .agg(ImmutableList.of( new IntegerLiteral(1), new IntegerLiteral(2), @@ -147,7 +147,7 @@ void testComplexGroupBy() { @Test void testOutOfRange() { LogicalPlan aggregate = new LogicalPlanBuilder( - new LogicalOlapScan(ObjectId.createGenerator().getNextId(), table)) + new LogicalOlapScan(RelationId.createGenerator().getNextId(), table)) .agg(ImmutableList.of( new StringLiteral("str"), k2, new IntegerLiteral(1), diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateUnnecessaryProjectTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateUnnecessaryProjectTest.java index de0462f3da46a9..548a7dd0689532 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateUnnecessaryProjectTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/EliminateUnnecessaryProjectTest.java @@ -19,6 +19,7 @@ import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.BooleanLiteral; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.types.IntegerType; @@ -59,7 +60,7 @@ void testEliminateNonTopUnnecessaryProject() { .build(); PlanChecker.from(MemoTestUtils.createConnectContext(), unnecessaryProject) - .applyTopDown(new EliminateUnnecessaryProject()) + .customRewrite(new EliminateUnnecessaryProject()) .matchesFromRoot(logicalFilter(logicalProject())); } @@ -70,7 +71,7 @@ void testEliminateTopUnnecessaryProject() { .build(); PlanChecker.from(MemoTestUtils.createConnectContext(), unnecessaryProject) - .applyTopDown(new EliminateUnnecessaryProject()) + .customRewrite(new EliminateUnnecessaryProject()) .matchesFromRoot(logicalOlapScan()); } @@ -81,19 +82,20 @@ void testNotEliminateTopProjectWhenOutputNotEquals() { .build(); PlanChecker.from(MemoTestUtils.createConnectContext(), necessaryProject) - .applyTopDown(new EliminateUnnecessaryProject()) + .customRewrite(new EliminateUnnecessaryProject()) .matchesFromRoot(logicalProject()); } @Test void testEliminateProjectWhenEmptyRelationChild() { - LogicalPlan unnecessaryProject = new LogicalPlanBuilder(new LogicalEmptyRelation(ImmutableList.of( - new SlotReference("k1", IntegerType.INSTANCE), - new SlotReference("k2", IntegerType.INSTANCE)))) + LogicalPlan unnecessaryProject = new LogicalPlanBuilder(new LogicalEmptyRelation(new RelationId(1), + ImmutableList.of( + new SlotReference("k1", IntegerType.INSTANCE), + new SlotReference("k2", IntegerType.INSTANCE)))) .project(ImmutableList.of(1, 0)) .build(); PlanChecker.from(MemoTestUtils.createConnectContext(), unnecessaryProject) - .applyTopDown(new EliminateUnnecessaryProject()) + .customRewrite(new EliminateUnnecessaryProject()) .matchesFromRoot(logicalEmptyRelation()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpressionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpressionTest.java index 22e1dc698abb87..e676caa37a8c98 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpressionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ExtractAndNormalizeWindowExpressionTest.java @@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.OrderExpression; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; import org.apache.doris.nereids.trees.expressions.functions.window.Rank; @@ -37,7 +38,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -54,7 +54,7 @@ public class ExtractAndNormalizeWindowExpressionTest implements MemoPatternMatch @BeforeAll public final void beforeAll() { - rStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, ImmutableList.of()); + rStudent = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student, ImmutableList.of()); } @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MergeProjectsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MergeProjectsTest.java index 40da31a3cf582a..e0cd9f622301e9 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MergeProjectsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/MergeProjectsTest.java @@ -19,6 +19,7 @@ import org.apache.doris.nereids.trees.expressions.Add; import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; @@ -28,7 +29,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -40,7 +40,7 @@ * MergeConsecutiveProjects ut */ public class MergeProjectsTest implements MemoPatternMatchSupported { - LogicalOlapScan score = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); + LogicalOlapScan score = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); @Test public void testMergeConsecutiveProjects() { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregateTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregateTest.java index e6a34daef72110..32f7b324f9af47 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregateTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/NormalizeAggregateTest.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.expressions.Multiply; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunction; import org.apache.doris.nereids.trees.expressions.functions.agg.Sum; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; @@ -36,7 +37,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; @@ -52,7 +52,7 @@ public class NormalizeAggregateTest implements MemoPatternMatchSupported { @BeforeAll public final void beforeAll() { - rStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, + rStudent = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student, ImmutableList.of()); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java index 7fd073ef074a89..e0244a80e3e8d6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PruneOlapScanTabletTest.java @@ -34,7 +34,7 @@ import org.apache.doris.nereids.trees.expressions.LessThanEqual; import org.apache.doris.nereids.trees.expressions.literal.DateLiteral; import org.apache.doris.nereids.trees.expressions.literal.Literal; -import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.util.MemoPatternMatchSupported; @@ -130,7 +130,7 @@ void testPruneOlapScanTablet(@Mocked OlapTable olapTable, } }; - LogicalOlapScan scan = new LogicalOlapScan(ObjectId.createGenerator().getNextId(), olapTable); + LogicalOlapScan scan = new LogicalOlapScan(RelationId.createGenerator().getNextId(), olapTable); GreaterThanEqual greaterThanEqual = new GreaterThanEqual(scan.getOutput().get(0), new DateLiteral("2019-08-22")); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughAggregationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughAggregationTest.java index 03f584afcf84fa..c6fe1f64208101 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughAggregationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughAggregationTest.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.expressions.LessThanEqual; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.functions.agg.Max; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; @@ -35,14 +36,13 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; import org.junit.jupiter.api.Test; public class PushdownFilterThroughAggregationTest implements MemoPatternMatchSupported { - private final LogicalOlapScan scan = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, + private final LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student, ImmutableList.of("")); /*- diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughSortTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughSortTest.java index b2861be62ccd84..0ccc12145ac5cf 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughSortTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughSortTest.java @@ -21,6 +21,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.GreaterThan; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; @@ -29,7 +30,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Test; @@ -37,7 +37,7 @@ import java.util.stream.Collectors; public class PushdownFilterThroughSortTest implements MemoPatternMatchSupported { - private final LogicalOlapScan scan = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, + private final LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student, ImmutableList.of("")); @Test diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughWindowTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughWindowTest.java index 8c5ac1ccf8ff7a..a53e8c6a053521 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughWindowTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownFilterThroughWindowTest.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.expressions.LessThanEqual; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.WindowFrame; import org.apache.doris.nereids.trees.expressions.functions.window.RowNumber; @@ -36,7 +37,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.qe.ConnectContext; import com.google.common.collect.ImmutableList; @@ -47,7 +47,7 @@ import java.util.List; public class PushdownFilterThroughWindowTest implements MemoPatternMatchSupported { - private final LogicalOlapScan scan = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student, + private final LogicalOlapScan scan = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student, ImmutableList.of("")); /*- diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownLimitTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownLimitTest.java index e19f991d8c76ac..f85882791e0b8b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownLimitTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownLimitTest.java @@ -25,6 +25,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.OrderExpression; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.WindowExpression; import org.apache.doris.nereids.trees.expressions.WindowFrame; import org.apache.doris.nereids.trees.expressions.functions.window.Rank; @@ -45,7 +46,6 @@ import org.apache.doris.nereids.util.MemoTestUtils; import org.apache.doris.nereids.util.PlanChecker; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.planner.OlapScanNode; import org.apache.doris.planner.PlanFragment; import org.apache.doris.qe.ConnectContext; @@ -64,8 +64,8 @@ import java.util.stream.Collectors; class PushdownLimitTest extends TestWithFeService implements MemoPatternMatchSupported { - private final LogicalOlapScan scanScore = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.score); - private Plan scanStudent = new LogicalOlapScan(RelationUtil.newRelationId(), PlanConstructor.student); + private final LogicalOlapScan scanScore = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.score); + private Plan scanStudent = new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), PlanConstructor.student); @Override protected void runBeforeAll() throws Exception { @@ -364,8 +364,8 @@ private Plan generatePlan(JoinType joinType, boolean hasProject) { LogicalJoin join = new LogicalJoin<>( joinType, joinConditions, - new LogicalOlapScan(((LogicalOlapScan) scanScore).getId(), PlanConstructor.score), - new LogicalOlapScan(((LogicalOlapScan) scanStudent).getId(), PlanConstructor.student) + new LogicalOlapScan(((LogicalOlapScan) scanScore).getRelationId(), PlanConstructor.score), + new LogicalOlapScan(((LogicalOlapScan) scanStudent).getRelationId(), PlanConstructor.student) ); if (hasProject) { diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java index 3c8be1594bd445..4ca19dedf70f4b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/stats/StatsCalculatorTest.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Or; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; import org.apache.doris.nereids.trees.plans.FakePlan; import org.apache.doris.nereids.trees.plans.GroupPlan; @@ -37,7 +38,6 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.qe.ConnectContext; import org.apache.doris.statistics.ColumnStatistic; import org.apache.doris.statistics.ColumnStatisticBuilder; @@ -252,7 +252,7 @@ public void testOlapScan(@Mocked ConnectContext context) { SlotReference slot1 = new SlotReference("c1", IntegerType.INSTANCE, true, qualifier); OlapTable table1 = PlanConstructor.newOlapTable(tableId1, "t1", 0); - LogicalOlapScan logicalOlapScan1 = (LogicalOlapScan) new LogicalOlapScan(RelationUtil.newRelationId(), table1, + LogicalOlapScan logicalOlapScan1 = (LogicalOlapScan) new LogicalOlapScan(StatementScopeIdGenerator.newRelationId(), table1, Collections.emptyList()).withGroupExprLogicalPropChildren(Optional.empty(), Optional.of(new LogicalProperties(() -> ImmutableList.of(slot1))), ImmutableList.of()); Group childGroup = newGroup(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanEqualsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanEqualsTest.java index ad9d1df91aae3a..fa333a0f16e3e7 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanEqualsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanEqualsTest.java @@ -28,12 +28,12 @@ import org.apache.doris.nereids.trees.expressions.ExprId; import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.SlotReference; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.functions.agg.AggregateParam; import org.apache.doris.nereids.trees.expressions.literal.Literal; import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; @@ -44,8 +44,6 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalQuickSort; import org.apache.doris.nereids.types.BigIntType; import org.apache.doris.nereids.util.ExpressionUtils; -import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableSet; @@ -122,17 +120,6 @@ public void testLogicalJoin(@Mocked Plan left, @Mocked Plan right) { Assertions.assertNotEquals(unexpected, actual); } - @Test - public void testLogicalOlapScan() { - LogicalOlapScan actual = PlanConstructor.newLogicalOlapScanWithSameId(0, "table", 0); - - LogicalOlapScan expected = PlanConstructor.newLogicalOlapScanWithSameId(0, "table", 0); - Assertions.assertEquals(expected, actual); - - LogicalOlapScan unexpected = PlanConstructor.newLogicalOlapScanWithSameId(1, "table", 0); - Assertions.assertNotEquals(unexpected, actual); - } - @Test public void testLogicalProject(@Mocked Plan child) { LogicalProject actual = new LogicalProject<>( @@ -250,24 +237,25 @@ public void testPhysicalOlapScan( @Mocked LogicalProperties logicalProperties, @Mocked OlapTable olapTable, @Mocked DistributionSpecHash distributionSpecHash) { + List selectedTabletId = Lists.newArrayList(); for (Partition partition : olapTable.getAllPartitions()) { selectedTabletId.addAll(partition.getBaseIndex().getTabletIdsInOrder()); } - ObjectId id = RelationUtil.newRelationId(); + RelationId id = StatementScopeIdGenerator.newRelationId(); PhysicalOlapScan actual = new PhysicalOlapScan(id, olapTable, Lists.newArrayList("a"), - olapTable.getBaseIndexId(), selectedTabletId, olapTable.getPartitionIds(), distributionSpecHash, + 1L, selectedTabletId, olapTable.getPartitionIds(), distributionSpecHash, PreAggStatus.on(), ImmutableList.of(), Optional.empty(), logicalProperties); PhysicalOlapScan expected = new PhysicalOlapScan(id, olapTable, Lists.newArrayList("a"), - olapTable.getBaseIndexId(), selectedTabletId, olapTable.getPartitionIds(), distributionSpecHash, + 1L, selectedTabletId, olapTable.getPartitionIds(), distributionSpecHash, PreAggStatus.on(), ImmutableList.of(), Optional.empty(), logicalProperties); Assertions.assertEquals(expected, actual); PhysicalOlapScan unexpected = new PhysicalOlapScan(id, olapTable, Lists.newArrayList("b"), - olapTable.getBaseIndexId(), selectedTabletId, olapTable.getPartitionIds(), distributionSpecHash, + 12345L, selectedTabletId, olapTable.getPartitionIds(), distributionSpecHash, PreAggStatus.on(), ImmutableList.of(), Optional.empty(), logicalProperties); Assertions.assertNotEquals(unexpected, actual); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanOutputTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanOutputTest.java index c0c8b155aad81b..65694a18aedbde 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanOutputTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanOutputTest.java @@ -20,11 +20,13 @@ import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.nereids.types.StringType; import org.apache.doris.nereids.util.PlanConstructor; import org.apache.doris.nereids.util.RelationUtil; +import org.apache.doris.statistics.Statistics; import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Assertions; @@ -50,7 +52,7 @@ public void testComputeOutput() { @Test public void testLazyComputeOutput() { // not throw exception when create new UnboundRelation - UnboundRelation relationPlan = new UnboundRelation(RelationUtil.newRelationId(), ImmutableList.of("a")); + UnboundRelation relationPlan = new UnboundRelation(StatementScopeIdGenerator.newRelationId(), ImmutableList.of("a")); try { // throw exception when getOutput diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/MemoTestUtils.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/MemoTestUtils.java index f5d7da7df818a7..8d3298f689dfa5 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/MemoTestUtils.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/MemoTestUtils.java @@ -81,7 +81,7 @@ public static CascadesContext createCascadesContext(ConnectContext connectContex public static CascadesContext createCascadesContext(StatementContext statementContext, Plan initPlan) { PhysicalProperties requestProperties = NereidsPlanner.buildInitRequireProperties(); - CascadesContext cascadesContext = CascadesContext.newRewriteContext( + CascadesContext cascadesContext = CascadesContext.initContext( statementContext, initPlan, requestProperties); cascadesContext.toMemo(); MemoValidator.validateInitState(cascadesContext.getMemo(), initPlan); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java index 613dbf17317664..66c855ae03df52 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanChecker.java @@ -30,7 +30,9 @@ import org.apache.doris.nereids.jobs.executor.Optimizer; import org.apache.doris.nereids.jobs.executor.Rewriter; import org.apache.doris.nereids.jobs.joinorder.JoinOrderJob; -import org.apache.doris.nereids.jobs.rewrite.CustomRewriteJob; +import org.apache.doris.nereids.jobs.rewrite.PlanTreeRewriteBottomUpJob; +import org.apache.doris.nereids.jobs.rewrite.PlanTreeRewriteTopDownJob; +import org.apache.doris.nereids.jobs.rewrite.RootPlanTreeRewriteJob; import org.apache.doris.nereids.memo.CopyInResult; import org.apache.doris.nereids.memo.Group; import org.apache.doris.nereids.memo.GroupExpression; @@ -64,12 +66,15 @@ import org.apache.doris.qe.OriginStatement; import com.google.common.base.Supplier; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; +import com.google.common.collect.Sets; import org.junit.jupiter.api.Assertions; import java.util.ArrayList; import java.util.List; import java.util.Optional; +import java.util.Set; import java.util.function.Consumer; /** @@ -79,8 +84,6 @@ public class PlanChecker { private final ConnectContext connectContext; private CascadesContext cascadesContext; - private Plan parsedPlan; - private PhysicalPlan physicalPlan; public PlanChecker(ConnectContext connectContext) { @@ -108,7 +111,7 @@ public static PlanChecker from(CascadesContext cascadesContext) { public PlanChecker checkParse(String sql, Consumer consumer) { PlanParseChecker checker = new PlanParseChecker(sql); consumer.accept(checker); - parsedPlan = checker.parsedSupplier.get(); + checker.parsedSupplier.get(); return this; } @@ -126,7 +129,12 @@ public PlanChecker analyze() { public PlanChecker analyze(Plan plan) { this.cascadesContext = MemoTestUtils.createCascadesContext(connectContext, plan); + Set originDisableRules = connectContext.getSessionVariable().getDisableNereidsRules(); + Set disableRuleWithAuth = Sets.newHashSet(originDisableRules); + disableRuleWithAuth.add(RuleType.RELATION_AUTHENTICATION.name()); + connectContext.getSessionVariable().setDisableNereidsRules(String.join(",", disableRuleWithAuth)); this.cascadesContext.newAnalyzer().analyze(); + connectContext.getSessionVariable().setDisableNereidsRules(String.join(",", originDisableRules)); this.cascadesContext.toMemo(); MemoValidator.validate(cascadesContext.getMemo()); return this; @@ -145,14 +153,12 @@ public PlanChecker customAnalyzer(Optional customTableResol return this; } - public PlanChecker setRewritePlanFromMemo() { - this.cascadesContext.setRewritePlan(this.cascadesContext.getMemo().copyOut()); - return this; - } - public PlanChecker customRewrite(CustomRewriter customRewriter) { - new CustomRewriteJob(() -> customRewriter, RuleType.TEST_REWRITE).execute(cascadesContext.getCurrentJobContext()); + Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, + ImmutableList.of(Rewriter.custom(RuleType.TEST_REWRITE, () -> customRewriter))) + .execute(); cascadesContext.toMemo(); + MemoValidator.validate(cascadesContext.getMemo()); return this; } @@ -160,14 +166,11 @@ public PlanChecker applyTopDown(RuleFactory ruleFactory) { return applyTopDown(ruleFactory.buildRules()); } - public PlanChecker applyTopDown(CustomRewriter customRewriter) { - cascadesContext.topDownRewrite(customRewriter); - MemoValidator.validate(cascadesContext.getMemo()); - return this; - } - public PlanChecker applyTopDown(List rule) { - cascadesContext.topDownRewrite(rule); + Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, + ImmutableList.of(new RootPlanTreeRewriteJob(rule, PlanTreeRewriteTopDownJob::new, true))) + .execute(); + cascadesContext.toMemo(); MemoValidator.validate(cascadesContext.getMemo()); return this; } @@ -178,7 +181,7 @@ public PlanChecker applyTopDown(List rule) { * @param patternMatcher the rule dsl, such as: logicalOlapScan().then(olapScan -> olapScan) * @return this checker, for call chaining of follow-up check */ - public PlanChecker applyTopDown(PatternMatcher patternMatcher) { + public PlanChecker applyTopDownInMemo(PatternMatcher patternMatcher) { cascadesContext.topDownRewrite(new OneRewriteRuleFactory() { @Override public Rule build() { @@ -190,7 +193,19 @@ public Rule build() { } public PlanChecker applyBottomUp(RuleFactory rule) { - cascadesContext.bottomUpRewrite(rule); + Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, + ImmutableList.of(Rewriter.bottomUp(rule))) + .execute(); + cascadesContext.toMemo(); + MemoValidator.validate(cascadesContext.getMemo()); + return this; + } + + public PlanChecker applyBottomUp(List rule) { + Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, + ImmutableList.of(new RootPlanTreeRewriteJob(rule, PlanTreeRewriteBottomUpJob::new, true))) + .execute(); + cascadesContext.toMemo(); MemoValidator.validate(cascadesContext.getMemo()); return this; } @@ -201,7 +216,7 @@ public PlanChecker applyBottomUp(RuleFactory rule) { * @param patternMatcher the rule dsl, such as: logicalOlapScan().then(olapScan -> olapScan) * @return this checker, for call chaining of follow-up check */ - public PlanChecker applyBottomUp(PatternMatcher patternMatcher) { + public PlanChecker applyBottomUpInMemo(PatternMatcher patternMatcher) { cascadesContext.bottomUpRewrite(new OneRewriteRuleFactory() { @Override public Rule build() { @@ -213,7 +228,7 @@ public Rule build() { } public PlanChecker rewrite() { - new Rewriter(cascadesContext).execute(); + Rewriter.getWholeTreeRewriter(cascadesContext).execute(); cascadesContext.toMemo(); return this; } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanConstructor.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanConstructor.java index 51948be52ba17c..9bb48837754e17 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanConstructor.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanConstructor.java @@ -25,7 +25,7 @@ import org.apache.doris.catalog.PartitionInfo; import org.apache.doris.catalog.Type; import org.apache.doris.common.IdGenerator; -import org.apache.doris.nereids.trees.plans.ObjectId; +import org.apache.doris.nereids.trees.plans.RelationId; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.thrift.TStorageType; @@ -39,7 +39,7 @@ public class PlanConstructor { public static final OlapTable score; public static final OlapTable course; - private static final IdGenerator RELATION_ID_GENERATOR = ObjectId.createGenerator(); + private static final IdGenerator RELATION_ID_GENERATOR = RelationId.createGenerator(); static { student = new OlapTable(0L, "student", @@ -108,12 +108,14 @@ public static LogicalOlapScan newLogicalOlapScan(long tableId, String tableName, ImmutableList.of("db")); } - public static LogicalOlapScan newLogicalOlapScanWithSameId(long tableId, String tableName, int hashColumn) { - return new LogicalOlapScan(ObjectId.createGenerator().getNextId(), - newOlapTable(tableId, tableName, hashColumn), ImmutableList.of("db")); + public static LogicalOlapScan newLogicalOlapScanWithSameId(long tableId, String tableName, + int hashColumn, List selectedPartitions) { + return new LogicalOlapScan(RelationId.createGenerator().getNextId(), + newOlapTable(tableId, tableName, hashColumn), ImmutableList.of("db"), + selectedPartitions, ImmutableList.of()); } - public static ObjectId getNextRelationId() { + public static RelationId getNextRelationId() { return RELATION_ID_GENERATOR.getNextId(); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanRewriter.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanRewriter.java index 67f6bc43ea3220..50b0789be9b1eb 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanRewriter.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/util/PlanRewriter.java @@ -17,59 +17,29 @@ package org.apache.doris.nereids.util; +import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.jobs.executor.Rewriter; import org.apache.doris.nereids.memo.Memo; -import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.rules.RuleFactory; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.OriginStatement; +import com.google.common.collect.ImmutableList; + /** * Utility to copy plan into {@link Memo} and apply rewrite rules. */ public class PlanRewriter { public static Plan bottomUpRewrite(Plan plan, ConnectContext connectContext, RuleFactory... rules) { - return bottomUpRewriteMemo(plan, connectContext, rules).copyOut(); - } - - public static Plan bottomUpRewrite(Plan plan, ConnectContext connectContext, Rule... rules) { - return bottomUpRewriteMemo(plan, connectContext, rules).copyOut(); - } - - public static Memo bottomUpRewriteMemo(Plan plan, ConnectContext connectContext, RuleFactory... rules) { - return new Memo(plan) - .newCascadesContext(new StatementContext(connectContext, new OriginStatement("", 0))) - .bottomUpRewrite(rules) - .getMemo(); - } - - public static Memo bottomUpRewriteMemo(Plan plan, ConnectContext connectContext, Rule... rules) { - return new Memo(plan) - .newCascadesContext(new StatementContext(connectContext, new OriginStatement("", 0))) - .bottomUpRewrite(rules) - .getMemo(); - } - - public static Plan topDownRewrite(Plan plan, ConnectContext connectContext, RuleFactory... rules) { - return topDownRewriteMemo(plan, connectContext, rules).copyOut(); - } - - public static Plan topDownRewrite(Plan plan, ConnectContext connectContext, Rule... rules) { - return topDownRewriteMemo(plan, connectContext, rules).copyOut(); - } - - public static Memo topDownRewriteMemo(Plan plan, ConnectContext connectContext, RuleFactory... rules) { - return new Memo(plan) - .newCascadesContext(new StatementContext(connectContext, new OriginStatement("", 0))) - .topDownRewrite(rules) - .getMemo(); - } - - public static Memo topDownRewriteMemo(Plan plan, ConnectContext connectContext, Rule... rules) { - return new Memo(plan) - .newCascadesContext(new StatementContext(connectContext, new OriginStatement("", 0))) - .topDownRewrite(rules) - .getMemo(); + CascadesContext cascadesContext = CascadesContext.initContext( + new StatementContext(connectContext, new OriginStatement("", 0)), + plan, + PhysicalProperties.GATHER); + Rewriter.getWholeTreeRewriterWithCustomJobs(cascadesContext, + ImmutableList.of(Rewriter.bottomUp(rules))).execute(); + return cascadesContext.getRewritePlan(); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java index cec9586e960adc..bb25f9c47d154a 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java +++ b/fe/fe-core/src/test/java/org/apache/doris/utframe/TestWithFeService.java @@ -45,6 +45,7 @@ import org.apache.doris.catalog.Database; import org.apache.doris.catalog.DiskInfo; import org.apache.doris.catalog.Env; +import org.apache.doris.catalog.InternalSchemaInitializer; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Replica; import org.apache.doris.catalog.Table; @@ -58,6 +59,7 @@ import org.apache.doris.common.util.SqlParserUtils; import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.StatementContext; +import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; import org.apache.doris.nereids.util.MemoTestUtils; @@ -82,6 +84,7 @@ import com.google.common.collect.ImmutableMap; import com.google.common.collect.Lists; import com.google.common.collect.Maps; +import com.google.common.collect.Sets; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.Assertions; import org.junit.jupiter.api.BeforeAll; @@ -101,6 +104,7 @@ import java.util.ConcurrentModificationException; import java.util.List; import java.util.Map; +import java.util.Set; import java.util.UUID; /** @@ -130,6 +134,7 @@ public final void beforeAll() throws Exception { connectContext = createDefaultCtx(); beforeCluster(); createDorisCluster(); + new InternalSchemaInitializer().start(); runBeforeAll(); } @@ -191,8 +196,13 @@ protected CascadesContext createCascadesContext(String sql) { } public LogicalPlan analyze(String sql) { + Set originDisableRules = connectContext.getSessionVariable().getDisableNereidsRules(); + Set disableRuleWithAuth = Sets.newHashSet(originDisableRules); + disableRuleWithAuth.add(RuleType.RELATION_AUTHENTICATION.name()); + connectContext.getSessionVariable().setDisableNereidsRules(String.join(",", disableRuleWithAuth)); CascadesContext cascadesContext = createCascadesContext(sql); cascadesContext.newAnalyzer().analyze(); + connectContext.getSessionVariable().setDisableNereidsRules(String.join(",", originDisableRules)); cascadesContext.toMemo(); return (LogicalPlan) cascadesContext.getRewritePlan(); } diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out index 838ba573be0cad..0a15a57002d99f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_1 -- -CteAnchor[cteId= ( CTEId#2=] ) ---CteProducer[cteId= ( CTEId#2=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] ------PhysicalDistribute --------hashAgg[LOCAL] @@ -26,7 +26,7 @@ CteAnchor[cteId= ( CTEId#2=] ) ----------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) ------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) --------------------PhysicalDistribute -----------------------CteConsumer[cteId= ( CTEId#2=] ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((cast(s_state as VARCHAR(*)) = 'SD')) @@ -37,5 +37,5 @@ CteAnchor[cteId= ( CTEId#2=] ) ------------------------hashAgg[LOCAL] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------CteConsumer[cteId= ( CTEId#2=] ) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 5e4621c1bd5e38..b97d80540ed59b 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_11 -- -CteAnchor[cteId= ( CTEId#4=] ) ---CteProducer[cteId= ( CTEId#4=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalUnion ------PhysicalProject --------hashAgg[GLOBAL] @@ -50,17 +50,17 @@ CteAnchor[cteId= ( CTEId#4=] ) ----------------PhysicalDistribute ------------------PhysicalProject --------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00)) -----------------------CteConsumer[cteId= ( CTEId#4=] ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject --------------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001)) -----------------------CteConsumer[cteId= ( CTEId#4=] ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002)) ---------------------CteConsumer[cteId= ( CTEId#4=] ) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------PhysicalDistribute --------------PhysicalProject ----------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w')) -------------------CteConsumer[cteId= ( CTEId#4=] ) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out index 4877f71f8c5652..04a155f26ccf04 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_14 -- -CteAnchor[cteId= ( CTEId#8=] ) ---CteProducer[cteId= ( CTEId#8=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----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 @@ -23,32 +23,19 @@ CteAnchor[cteId= ( CTEId#8=] ) --------------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 -------------PhysicalOlapScan[item] ---CteAnchor[cteId= ( CTEId#10=] ) -----CteProducer[cteId= ( CTEId#10=] ) +--------------------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] +--PhysicalCteAnchor ( cteId=CTEId#1 ) +----PhysicalCteProducer ( cteId=CTEId#1 ) ------hashAgg[GLOBAL] --------PhysicalDistribute ----------hashAgg[LOCAL] @@ -98,7 +85,7 @@ CteAnchor[cteId= ( CTEId#8=] ) ----------------------------------PhysicalProject ------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk) --------------------------------------PhysicalDistribute -----------------------------------------CteConsumer[cteId= ( CTEId#8=] ) +----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ----------------------------------------PhysicalDistribute ------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) @@ -114,7 +101,7 @@ CteAnchor[cteId= ( CTEId#8=] ) --------------------------PhysicalDistribute ----------------------------PhysicalAssertNumRows ------------------------------PhysicalDistribute ---------------------------------CteConsumer[cteId= ( CTEId#10=] ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------PhysicalProject ------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) --------------------------PhysicalProject @@ -124,7 +111,7 @@ CteAnchor[cteId= ( CTEId#8=] ) ----------------------------------PhysicalProject ------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk) --------------------------------------PhysicalDistribute -----------------------------------------CteConsumer[cteId= ( CTEId#8=] ) +----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ----------------------------------------PhysicalDistribute ------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) @@ -140,7 +127,7 @@ CteAnchor[cteId= ( CTEId#8=] ) --------------------------PhysicalDistribute ----------------------------PhysicalAssertNumRows ------------------------------PhysicalDistribute ---------------------------------CteConsumer[cteId= ( CTEId#10=] ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------PhysicalProject ------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) --------------------------PhysicalProject @@ -150,7 +137,7 @@ CteAnchor[cteId= ( CTEId#8=] ) ----------------------------------PhysicalProject ------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk) --------------------------------------PhysicalDistribute -----------------------------------------CteConsumer[cteId= ( CTEId#8=] ) +----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ----------------------------------------PhysicalDistribute ------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) @@ -166,5 +153,5 @@ CteAnchor[cteId= ( CTEId#8=] ) --------------------------PhysicalDistribute ----------------------------PhysicalAssertNumRows ------------------------------PhysicalDistribute ---------------------------------CteConsumer[cteId= ( CTEId#10=] ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) 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 13e0f2f0e9c2e6..4d4ce2d58c8081 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_2 -- -CteAnchor[cteId= ( CTEId#4=] ) ---CteProducer[cteId= ( CTEId#4=] ) +PhysicalCteAnchor ( cteId=CTEId#1 ) +--PhysicalCteProducer ( cteId=CTEId#1 ) ----hashAgg[GLOBAL] ------PhysicalDistribute --------hashAgg[LOCAL] @@ -27,7 +27,7 @@ CteAnchor[cteId= ( CTEId#4=] ) ----------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq2) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------CteConsumer[cteId= ( CTEId#4=] ) +----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter((date_dim.d_year = 1999)) @@ -37,7 +37,7 @@ CteAnchor[cteId= ( CTEId#4=] ) ----------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq1) ------------------PhysicalDistribute --------------------PhysicalProject -----------------------CteConsumer[cteId= ( CTEId#4=] ) +----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter((date_dim.d_year = 1998)) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out index 9f4a6bcf94965c..1deef474c9d053 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_23 -- -CteAnchor[cteId= ( CTEId#1=] ) ---CteProducer[cteId= ( CTEId#1=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------filter((cnt > 4)) --------hashAgg[GLOBAL] @@ -19,8 +19,8 @@ CteAnchor[cteId= ( CTEId#1=] ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[item] ---CteAnchor[cteId= ( CTEId#4=] ) -----CteProducer[cteId= ( CTEId#4=] ) +--PhysicalCteAnchor ( cteId=CTEId#2 ) +----PhysicalCteProducer ( cteId=CTEId#2 ) ------PhysicalProject --------NestedLoopJoin[INNER_JOIN](cast(ssales as DOUBLE) > cast(((cast(95 as DECIMALV3(8, 5)) / 100.0) * tpcds_cmax) as DOUBLE)) ----------hashAgg[GLOBAL] @@ -65,7 +65,7 @@ CteAnchor[cteId= ( CTEId#1=] ) ------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = frequent_ss_items.item_sk) --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------CteConsumer[cteId= ( CTEId#1=] ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------hashJoin[LEFT_SEMI_JOIN](catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk) @@ -79,12 +79,12 @@ CteAnchor[cteId= ( CTEId#1=] ) ------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------CteConsumer[cteId= ( CTEId#4=] ) +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) ----------------PhysicalProject ------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = frequent_ss_items.item_sk) --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------CteConsumer[cteId= ( CTEId#1=] ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------hashJoin[LEFT_SEMI_JOIN](web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk) @@ -98,5 +98,5 @@ CteAnchor[cteId= ( CTEId#1=] ) ------------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------CteConsumer[cteId= ( CTEId#4=] ) +------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out index 56bad7457e0d7e..eebe6647279d9d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_24 -- -CteAnchor[cteId= ( CTEId#0=] ) ---CteProducer[cteId= ( CTEId#0=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------hashAgg[GLOBAL] --------PhysicalDistribute @@ -42,7 +42,7 @@ CteAnchor[cteId= ( CTEId#0=] ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter((cast(i_color as VARCHAR(*)) = 'beige')) -------------------------CteConsumer[cteId= ( CTEId#0=] ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------PhysicalDistribute --------------PhysicalAssertNumRows ----------------PhysicalProject @@ -51,5 +51,5 @@ CteAnchor[cteId= ( CTEId#0=] ) ----------------------hashAgg[LOCAL] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------CteConsumer[cteId= ( CTEId#0=] ) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 7338d341dc0856..7741e0c45cd144 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_30 -- -CteAnchor[cteId= ( CTEId#2=] ) ---CteProducer[cteId= ( CTEId#2=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] ------PhysicalDistribute --------hashAgg[LOCAL] @@ -26,7 +26,7 @@ CteAnchor[cteId= ( CTEId#2=] ) ----------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) ------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) --------------PhysicalDistribute -----------------CteConsumer[cteId= ( CTEId#2=] ) +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) ------------------PhysicalProject @@ -42,5 +42,5 @@ CteAnchor[cteId= ( CTEId#2=] ) --------------------hashAgg[LOCAL] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------CteConsumer[cteId= ( CTEId#2=] ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out index 12a7b7db3138b2..dd60ee2d41baf1 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_31 -- -CteAnchor[cteId= ( CTEId#6=] ) ---CteProducer[cteId= ( CTEId#6=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------hashAgg[GLOBAL] --------PhysicalDistribute @@ -19,8 +19,8 @@ CteAnchor[cteId= ( CTEId#6=] ) ----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalOlapScan[customer_address] ---CteAnchor[cteId= ( CTEId#7=] ) -----CteProducer[cteId= ( CTEId#7=] ) +--PhysicalCteAnchor ( cteId=CTEId#1 ) +----PhysicalCteProducer ( cteId=CTEId#1 ) ------PhysicalProject --------hashAgg[GLOBAL] ----------PhysicalDistribute @@ -47,30 +47,30 @@ CteAnchor[cteId= ( CTEId#6=] ) --------------PhysicalDistribute ----------------PhysicalProject ------------------filter((ws3.d_year = 2000)(ws3.d_qoy = 3)) ---------------------CteConsumer[cteId= ( CTEId#7=] ) +--------------------PhysicalCteConsumer ( cteId=CTEId#1 ) --------------PhysicalProject ----------------hashJoin[INNER_JOIN](ss2.ca_county = ss3.ca_county) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------filter((ss3.d_year = 2000)(ss3.d_qoy = 3)) -------------------------CteConsumer[cteId= ( CTEId#6=] ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------hashJoin[INNER_JOIN](ws1.ca_county = ws2.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) --------------------hashJoin[INNER_JOIN](ss1.ca_county = ws1.ca_county) ----------------------hashJoin[INNER_JOIN](ss1.ca_county = ss2.ca_county) ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------filter((ss1.d_year = 2000)(ss1.d_qoy = 1)) -------------------------------CteConsumer[cteId= ( CTEId#6=] ) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------filter((ss2.d_year = 2000)(ss2.d_qoy = 2)) -------------------------------CteConsumer[cteId= ( CTEId#6=] ) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------filter((ws1.d_year = 2000)(ws1.d_qoy = 1)) -----------------------------CteConsumer[cteId= ( CTEId#7=] ) +----------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((ws2.d_qoy = 2)(ws2.d_year = 2000)) ---------------------------CteConsumer[cteId= ( CTEId#7=] ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out index 7465afe902fbd2..bd6e7a2a90ff73 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_39 -- -CteAnchor[cteId= ( CTEId#3=] ) ---CteProducer[cteId= ( CTEId#3=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------filter((CASE WHEN (mean = 0.0) THEN 0.0 ELSE (stdev / mean) END > 1.0)) --------hashAgg[GLOBAL] @@ -30,9 +30,9 @@ CteAnchor[cteId= ( CTEId#3=] ) ------------PhysicalDistribute --------------PhysicalProject ----------------filter((inv1.d_moy = 1)) -------------------CteConsumer[cteId= ( CTEId#3=] ) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------PhysicalDistribute --------------PhysicalProject ----------------filter((inv2.d_moy = 2)) -------------------CteConsumer[cteId= ( CTEId#3=] ) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out index 897f72cdd629e8..1ea2eae2bcf004 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_4 -- -CteAnchor[cteId= ( CTEId#6=] ) ---CteProducer[cteId= ( CTEId#6=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalUnion ------PhysicalProject --------hashAgg[GLOBAL] @@ -75,25 +75,25 @@ CteAnchor[cteId= ( CTEId#6=] ) --------------------------PhysicalDistribute ----------------------------PhysicalProject ------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's')) ---------------------------------CteConsumer[cteId= ( CTEId#6=] ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------PhysicalProject ------------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c')) ---------------------------------CteConsumer[cteId= ( CTEId#6=] ) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------PhysicalDistribute --------------------------PhysicalProject ----------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000)) -------------------------------CteConsumer[cteId= ( CTEId#6=] ) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((t_c_secyear.sale_type = 'c')(t_c_secyear.dyear = 2000)) ---------------------------CteConsumer[cteId= ( CTEId#6=] ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject --------------------filter((t_w_firstyear.dyear = 1999)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000)) -----------------------CteConsumer[cteId= ( CTEId#6=] ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------PhysicalDistribute --------------PhysicalProject ----------------filter((t_w_secyear.sale_type = 'w')(t_w_secyear.dyear = 2000)) -------------------CteConsumer[cteId= ( CTEId#6=] ) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 c6f3c083e0651f..7b27c69128a9b9 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_47 -- -CteAnchor[cteId= ( CTEId#0=] ) ---CteProducer[cteId= ( CTEId#0=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------PhysicalWindow --------PhysicalQuickSort @@ -37,15 +37,15 @@ CteAnchor[cteId= ( CTEId#0=] ) ------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) --------------PhysicalDistribute ----------------PhysicalProject -------------------CteConsumer[cteId= ( CTEId#0=] ) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------CteConsumer[cteId= ( CTEId#0=] ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 2001)(v2.avg_monthly_sales > 0.0000)) ---------------------------CteConsumer[cteId= ( CTEId#0=] ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 72a23fe3e82503..e6002e5a2d6c48 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 @@ -53,7 +53,7 @@ PhysicalTopN --------------------------------------------------------------------------PhysicalOlapScan[item] ----------------------------------------------------------------PhysicalDistribute ------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------filter((date_dim.d_moy = 5)(date_dim.d_year = 1998)) +--------------------------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) ----------------------------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject @@ -65,7 +65,7 @@ PhysicalTopN ------------------------------------------PhysicalDistribute --------------------------------------------hashAgg[LOCAL] ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_moy = 5)(date_dim.d_year = 1998)) +------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) --------------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalAssertNumRows 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 ed85d04d8c3a0c..0e13df083b4ceb 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_57 -- -CteAnchor[cteId= ( CTEId#0=] ) ---CteProducer[cteId= ( CTEId#0=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------PhysicalWindow --------PhysicalQuickSort @@ -37,15 +37,15 @@ CteAnchor[cteId= ( CTEId#0=] ) ------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1)) --------------PhysicalDistribute ----------------PhysicalProject -------------------CteConsumer[cteId= ( CTEId#0=] ) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1)) --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------CteConsumer[cteId= ( CTEId#0=] ) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 1999)(v2.avg_monthly_sales > 0.0000)) ---------------------------CteConsumer[cteId= ( CTEId#0=] ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out index 9af8591c8f6e43..89bed6d2465ef1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_59 -- -CteAnchor[cteId= ( CTEId#4=] ) ---CteProducer[cteId= ( CTEId#4=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------hashAgg[GLOBAL] --------PhysicalDistribute @@ -20,17 +20,25 @@ CteAnchor[cteId= ( CTEId#4=] ) ----------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) ------------PhysicalDistribute --------------PhysicalProject -----------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) -------------------PhysicalOlapScan[date_dim] -------------PhysicalDistribute ---------------hashJoin[INNER_JOIN](y.s_store_id1 = x.s_store_id2)(wss.ss_store_sk = store.s_store_sk) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52)) +----------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) +--------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute --------------------PhysicalProject -----------------------CteConsumer[cteId= ( CTEId#4=] ) +----------------------PhysicalOlapScan[store] +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out index 4f9cb262cf3f38..afceaaa1cd27fe 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_74 -- -CteAnchor[cteId= ( CTEId#4=] ) ---CteProducer[cteId= ( CTEId#4=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalUnion ------PhysicalProject --------hashAgg[GLOBAL] @@ -58,9 +58,17 @@ CteAnchor[cteId= ( CTEId#4=] ) ----------------PhysicalDistribute ------------------PhysicalProject --------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.0)) -----------------------CteConsumer[cteId= ( CTEId#4=] ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject --------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000)) -----------------------CteConsumer[cteId= ( CTEId#4=] ) +----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------PhysicalDistribute +--------------PhysicalProject +----------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w')) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out index 0ad25cc5ec408c..09a72797851f63 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_75 -- -CteAnchor[cteId= ( CTEId#3=] ) ---CteProducer[cteId= ( CTEId#3=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] ------PhysicalDistribute --------hashAgg[LOCAL] @@ -68,8 +68,8 @@ CteAnchor[cteId= ( CTEId#3=] ) ----------hashJoin[INNER_JOIN](curr_yr.i_brand_id = prev_yr.i_brand_id)(curr_yr.i_class_id = prev_yr.i_class_id)(curr_yr.i_category_id = prev_yr.i_category_id)(curr_yr.i_manufact_id = prev_yr.i_manufact_id)((cast(cast(sales_cnt as DECIMALV3(17, 2)) as DECIMALV3(23, 8)) / cast(sales_cnt as DECIMALV3(17, 2))) < 0.900000) ------------PhysicalDistribute --------------filter((curr_yr.d_year = 1999)) -----------------CteConsumer[cteId= ( CTEId#3=] ) +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------PhysicalDistribute --------------filter((prev_yr.d_year = 1998)) -----------------CteConsumer[cteId= ( CTEId#3=] ) +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) 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 bb72d81784ebc8..11524e0f10bfcc 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 @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_81 -- -CteAnchor[cteId= ( CTEId#2=] ) ---CteProducer[cteId= ( CTEId#2=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----hashAgg[GLOBAL] ------PhysicalDistribute --------hashAgg[LOCAL] @@ -26,7 +26,7 @@ CteAnchor[cteId= ( CTEId#2=] ) ----------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) ------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) --------------PhysicalDistribute -----------------CteConsumer[cteId= ( CTEId#2=] ) +----------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) ------------------PhysicalProject @@ -42,5 +42,5 @@ CteAnchor[cteId= ( CTEId#2=] ) --------------------hashAgg[LOCAL] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------CteConsumer[cteId= ( CTEId#2=] ) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out index 014535c50dbb18..410beed37167a1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out @@ -1,7 +1,7 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_95 -- -CteAnchor[cteId= ( CTEId#3=] ) ---CteProducer[cteId= ( CTEId#3=] ) +PhysicalCteAnchor ( cteId=CTEId#0 ) +--PhysicalCteProducer ( cteId=CTEId#0 ) ----PhysicalProject ------hashJoin[INNER_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) --------PhysicalDistribute @@ -27,7 +27,7 @@ CteAnchor[cteId= ( CTEId#3=] ) ------------------------hashJoin[INNER_JOIN](web_returns.wr_order_number = ws_wh.ws_order_number) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------CteConsumer[cteId= ( CTEId#3=] ) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------PhysicalProject ------------------------------PhysicalOlapScan[web_returns] @@ -35,7 +35,7 @@ CteAnchor[cteId= ( CTEId#3=] ) ------------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws_wh.ws_order_number) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------CteConsumer[cteId= ( CTEId#3=] ) +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) ------------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) From 0cb96c78d9c9ebf0ed4e9a89fbd4baeed83ee1a9 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Tue, 25 Jul 2023 17:51:49 +0800 Subject: [PATCH 6/8] [refactor](Nereids) add sink interface and abstract class (#22150) 1. add trait Sink 2. add abstract class LogicalSink and PhysicalSink 3. replace some sink visitor by visitLogicalSink and visitPhysicalSink --- .../analyzer/UnboundOlapTableSink.java | 6 +- .../processor/post/FragmentProcessor.java | 2 +- .../post/RuntimeFilterGenerator.java | 14 +- .../processor/post/RuntimeFilterPruner.java | 2 +- .../ChildOutputPropertyDeriver.java | 21 +- .../rules/rewrite/AddDefaultLimit.java | 16 +- .../nereids/rules/rewrite/ColumnPruning.java | 12 +- .../rules/rewrite/PullUpCteAnchor.java | 14 +- .../doris/nereids/stats/StatsCalculator.java | 20 +- .../trees/copier/LogicalPlanDeepCopier.java | 17 +- .../nereids/trees/plans/algebra/Sink.java | 24 ++ .../trees/plans/logical/LogicalFileSink.java | 4 +- .../plans/logical/LogicalOlapTableSink.java | 5 +- .../trees/plans/logical/LogicalSink.java | 39 +++ .../plans/logical/LogicalSubQueryAlias.java | 2 +- .../plans/physical/PhysicalFileSink.java | 4 +- .../plans/physical/PhysicalOlapTableSink.java | 4 +- .../plans/physical/PhysicalRelation.java | 2 +- .../trees/plans/physical/PhysicalSink.java | 52 +++ .../trees/plans/visitor/CommandVisitor.java | 1 + .../trees/plans/visitor/PlanVisitor.java | 303 ++++++------------ .../trees/plans/visitor/RelationVisitor.java | 142 ++++++++ .../trees/plans/visitor/SinkVisitor.java | 73 +++++ 23 files changed, 490 insertions(+), 289 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Sink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/RelationVisitor.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java index 433684495105f6..9bf097f994ccf6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java @@ -24,7 +24,8 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; -import org.apache.doris.nereids.trees.plans.logical.LogicalUnary; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -37,7 +38,8 @@ /** * Represent an olap table sink plan node that has not been bound. */ -public class UnboundOlapTableSink extends LogicalUnary implements Unbound { +public class UnboundOlapTableSink extends LogicalSink implements Unbound, Sink { + private final List nameParts; private final List colNames; private final List hints; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/FragmentProcessor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/FragmentProcessor.java index e07b72b2f7d5c0..750ea8024f76ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/FragmentProcessor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/FragmentProcessor.java @@ -45,7 +45,7 @@ public PhysicalHashJoin visitPhysicalHashJoin(PhysicalHashJoin project } @Override - public PhysicalRelation visitPhysicalScan(PhysicalRelation scan, CascadesContext context) { + public Plan visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, CascadesContext context) { + // TODO: OneRowRelation will be translated to union. Union node cannot apply runtime filter now + // so, just return itself now, until runtime filter could apply on any node. + return oneRowRelation; + } + + @Override + public PhysicalRelation visitPhysicalRelation(PhysicalRelation relation, CascadesContext context) { // add all the slots in map. RuntimeFilterContext ctx = context.getRuntimeFilterContext(); - scan.getOutput().forEach(slot -> ctx.getAliasTransferMap().put(slot, Pair.of(scan, slot))); - return scan; + relation.getOutput().forEach(slot -> ctx.getAliasTransferMap().put(slot, Pair.of(relation, slot))); + return relation; } private long getBuildSideNdv(PhysicalHashJoin join, EqualTo equalTo) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java index eb31e89eb94f73..aa1f10aa47849e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/post/RuntimeFilterPruner.java @@ -129,7 +129,7 @@ public PhysicalFilter visitPhysicalFilter(PhysicalFilter filter, } @Override - public PhysicalRelation visitPhysicalScan(PhysicalRelation scan, CascadesContext context) { + public PhysicalRelation visitPhysicalRelation(PhysicalRelation scan, CascadesContext context) { RuntimeFilterContext rfCtx = context.getRuntimeFilterContext(); List slots = rfCtx.getTargetOnOlapScanNodeMap().get(scan.getRelationId()); if (slots != null) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java index 6672ecfe08539e..2ebc3acc5fe946 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/ChildOutputPropertyDeriver.java @@ -38,7 +38,6 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalEsScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalFileScan; -import org.apache.doris.nereids.trees.plans.physical.PhysicalFileSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; import org.apache.doris.nereids.trees.plans.physical.PhysicalGenerate; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; @@ -47,12 +46,12 @@ 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.PhysicalOlapScan; -import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalPartitionTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.trees.plans.physical.PhysicalRepeat; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; @@ -92,26 +91,20 @@ public PhysicalProperties getOutputProperties(GroupExpression groupExpression) { return groupExpression.getPlan().accept(this, new PlanContext(groupExpression)); } + @Override + public PhysicalProperties visit(Plan plan, PlanContext context) { + return PhysicalProperties.ANY; + } + /* ******************************************************************************************** * sink Node, in lexicographical order * ******************************************************************************************** */ @Override - public PhysicalProperties visitPhysicalFileSink(PhysicalFileSink fileSink, PlanContext context) { + public PhysicalProperties visitPhysicalSink(PhysicalSink physicalSink, PlanContext context) { return PhysicalProperties.GATHER; } - @Override - public PhysicalProperties visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, - PlanContext context) { - return PhysicalProperties.GATHER; - } - - @Override - public PhysicalProperties visit(Plan plan, PlanContext context) { - return PhysicalProperties.ANY; - } - /* ******************************************************************************************** * Leaf Plan Node, in lexicographical order * ******************************************************************************************** */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java index 1b5fd7314b9d07..93cbf2f7940489 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java @@ -22,9 +22,8 @@ import org.apache.doris.nereids.trees.plans.LimitPhase; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; -import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; @@ -63,16 +62,9 @@ public Plan visitLogicalCTEAnchor(LogicalCTEAnchor olapTableSink, - StatementContext context) { - Plan child = olapTableSink.child().accept(this, context); - return olapTableSink.withChildren(child); - } - - @Override - public Plan visitLogicalFileSink(LogicalFileSink fileSink, StatementContext context) { - Plan child = fileSink.child().accept(this, context); - return fileSink.withChildren(child); + public Plan visitLogicalSink(LogicalSink logicalSink, StatementContext context) { + Plan child = logicalSink.child().accept(this, context); + return logicalSink.withChildren(child); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java index 2fc17a717d06f8..85e0f84524caac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/ColumnPruning.java @@ -28,11 +28,10 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; -import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalIntersect; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.trees.plans.logical.OutputPrunable; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; @@ -162,13 +161,8 @@ public Plan visitLogicalIntersect(LogicalIntersect intersect, PruneContext conte } @Override - public Plan visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, PruneContext context) { - return skipPruneThisAndFirstLevelChildren(olapTableSink); - } - - @Override - public Plan visitLogicalFileSink(LogicalFileSink fileSink, PruneContext context) { - return skipPruneThisAndFirstLevelChildren(fileSink); + public Plan visitLogicalSink(LogicalSink logicalSink, PruneContext context) { + return skipPruneThisAndFirstLevelChildren(logicalSink); } // the backend not support filter(project(agg)), so we can not prune the key set in the agg, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java index 027a20b3db43ca..356333e4f69907 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java @@ -21,8 +21,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; -import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; @@ -77,14 +76,7 @@ public LogicalCTEProducer visitLogicalCTEProducer(LogicalCTEProducer olapTableSink, - List> producers) { - return olapTableSink.withChildren(rewriteRoot(olapTableSink.child(), producers)); - } - - @Override - public Plan visitLogicalFileSink(LogicalFileSink fileSink, - List> producers) { - return fileSink.withChildren(rewriteRoot(fileSink.child(), producers)); + public Plan visitLogicalSink(LogicalSink logicalSink, List> producers) { + return logicalSink.withChildren(rewriteRoot(logicalSink.child(), producers)); } } 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 a7cf8648fea3ce..472e4306933870 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 @@ -61,7 +61,6 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalEsScan; import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; -import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalGenerate; import org.apache.doris.nereids.trees.plans.logical.LogicalIntersect; @@ -69,12 +68,12 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; import org.apache.doris.nereids.trees.plans.logical.LogicalSchemaScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; @@ -89,7 +88,6 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalEsScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalExcept; import org.apache.doris.nereids.trees.plans.physical.PhysicalFileScan; -import org.apache.doris.nereids.trees.plans.physical.PhysicalFileSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; import org.apache.doris.nereids.trees.plans.physical.PhysicalGenerate; import org.apache.doris.nereids.trees.plans.physical.PhysicalHashAggregate; @@ -99,13 +97,13 @@ 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.PhysicalOlapScan; -import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalPartitionTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.trees.plans.physical.PhysicalQuickSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalRepeat; import org.apache.doris.nereids.trees.plans.physical.PhysicalSchemaScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate; import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; @@ -242,12 +240,7 @@ we record the lowest expression cost as group cost to avoid missing this group. } @Override - public Statistics visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, Void context) { - return groupExpression.childStatistics(0); - } - - @Override - public Statistics visitLogicalFileSink(LogicalFileSink fileSink, Void context) { + public Statistics visitLogicalSink(LogicalSink logicalSink, Void context) { return groupExpression.childStatistics(0); } @@ -379,12 +372,7 @@ public Statistics visitLogicalWindow(LogicalWindow window, Void } @Override - public Statistics visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, Void context) { - return groupExpression.childStatistics(0); - } - - @Override - public Statistics visitPhysicalFileSink(PhysicalFileSink fileSink, Void context) { + public Statistics visitPhysicalSink(PhysicalSink physicalSink, Void context) { return groupExpression.childStatistics(0); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java index 9da4690390ee5b..400d27e71aa9f2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/copier/LogicalPlanDeepCopier.java @@ -39,7 +39,6 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalEsScan; import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; -import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalGenerate; import org.apache.doris.nereids.trees.plans.logical.LogicalHaving; @@ -48,13 +47,13 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; 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.LogicalRepeat; import org.apache.doris.nereids.trees.plans.logical.LogicalSchemaScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; @@ -375,17 +374,9 @@ public Plan visitLogicalWindow(LogicalWindow window, DeepCopierC } @Override - public Plan visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, - DeepCopierContext context) { - Plan child = olapTableSink.child().accept(this, context); - return new LogicalOlapTableSink<>(olapTableSink.getDatabase(), olapTableSink.getTargetTable(), - olapTableSink.getCols(), olapTableSink.getPartitionIds(), child); - } - - @Override - public Plan visitLogicalFileSink(LogicalFileSink fileSink, DeepCopierContext context) { - Plan child = fileSink.child().accept(this, context); - return fileSink.withChildren(child); + public Plan visitLogicalSink(LogicalSink logicalSink, DeepCopierContext context) { + Plan child = logicalSink.child().accept(this, context); + return logicalSink.withChildren(child); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Sink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Sink.java new file mode 100644 index 00000000000000..e662c5882888d6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/algebra/Sink.java @@ -0,0 +1,24 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.algebra; + +/** + * traits for all sink + */ +public interface Sink { +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileSink.java index 4523ee881bb2e5..89918d231744d1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalFileSink.java @@ -23,6 +23,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import com.google.common.base.Preconditions; @@ -37,7 +38,8 @@ /** * logicalFileSink for select into outfile */ -public class LogicalFileSink extends LogicalUnary { +public class LogicalFileSink extends LogicalSink implements Sink { + private final String filePath; private final String format; private final Map properties; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java index b1c514f2541ebd..4ee8814f2c778b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; @@ -39,7 +40,7 @@ /** * logical olap table sink for insert command */ -public class LogicalOlapTableSink extends LogicalUnary { +public class LogicalOlapTableSink extends LogicalSink implements Sink { // bound data sink private Database database; private OlapTable targetTable; @@ -71,7 +72,7 @@ public LogicalOlapTableSink(Database database, OlapTable targetTable, List children) { Preconditions.checkArgument(children.size() == 1, "LogicalOlapTableSink only accepts one child"); return new LogicalOlapTableSink<>(database, targetTable, cols, partitionIds, isPartialUpdate, - groupExpression, Optional.of(getLogicalProperties()), children.get(0)); + Optional.empty(), Optional.empty(), children.get(0)); } public Database getDatabase() { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSink.java new file mode 100644 index 00000000000000..fd98c29a05ad6d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSink.java @@ -0,0 +1,39 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.logical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; + +import java.util.Optional; + +/** abstract logical sink */ +public abstract class LogicalSink extends LogicalUnary { + + public LogicalSink(PlanType type, CHILD_TYPE child) { + super(type, child); + } + + public LogicalSink(PlanType type, + Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + super(type, groupExpression, logicalProperties, child); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java index ef8588b1fe6117..5bef42f4f2c65b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalSubQueryAlias.java @@ -132,7 +132,7 @@ public LogicalSubQueryAlias withChildren(List children) { @Override public R accept(PlanVisitor visitor, C context) { - return visitor.visitSubQueryAlias(this, context); + return visitor.visitLogicalSubQueryAlias(this, context); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFileSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFileSink.java index b2849efcf4233b..8d33bc367cc26f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFileSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalFileSink.java @@ -24,6 +24,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.statistics.Statistics; @@ -38,7 +39,8 @@ /** * physicalFileSink for select into outfile */ -public class PhysicalFileSink extends PhysicalUnary { +public class PhysicalFileSink extends PhysicalSink implements Sink { + private final String filePath; private final String format; private final Map properties; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java index 93bf78719d35ea..14ae05a61e44a8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java @@ -29,6 +29,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; @@ -47,7 +48,8 @@ /** * physical olap table sink for insert command */ -public class PhysicalOlapTableSink extends PhysicalUnary { +public class PhysicalOlapTableSink extends PhysicalSink implements Sink { + private final Database database; private final OlapTable targetTable; private final List cols; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRelation.java index 49b42bb02a344d..cb1750d8c15602 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRelation.java @@ -78,7 +78,7 @@ public int hashCode() { @Override public R accept(PlanVisitor visitor, C context) { - return visitor.visitPhysicalScan(this, context); + return visitor.visitPhysicalRelation(this, context); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSink.java new file mode 100644 index 00000000000000..be6837b0a5bce9 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalSink.java @@ -0,0 +1,52 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.physical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.statistics.Statistics; + +import org.jetbrains.annotations.Nullable; + +import java.util.Optional; + +/** abstract physical sink */ +public abstract class PhysicalSink extends PhysicalUnary { + + public PhysicalSink(PlanType type, + LogicalProperties logicalProperties, CHILD_TYPE child) { + super(type, logicalProperties, child); + } + + public PhysicalSink(PlanType type, + Optional groupExpression, + LogicalProperties logicalProperties, CHILD_TYPE child) { + super(type, groupExpression, logicalProperties, child); + } + + public PhysicalSink(PlanType type, + Optional groupExpression, + LogicalProperties logicalProperties, + @Nullable PhysicalProperties physicalProperties, + Statistics statistics, CHILD_TYPE child) { + super(type, groupExpression, logicalProperties, physicalProperties, statistics, child); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java index 264598d9284d14..fbdf1085774ea8 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java @@ -26,6 +26,7 @@ /** CommandVisitor. */ public interface CommandVisitor { + R visitCommand(Command command, C context); default R visitExplainCommand(ExplainCommand explain, C context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java index 1beff0f1230a52..c1077f7a7e7c12 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java @@ -17,10 +17,6 @@ package org.apache.doris.nereids.trees.plans.visitor; -import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; -import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; -import org.apache.doris.nereids.analyzer.UnboundRelation; -import org.apache.doris.nereids.analyzer.UnboundTVFRelation; import org.apache.doris.nereids.trees.plans.GroupPlan; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.commands.Command; @@ -32,31 +28,22 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalCTEConsumer; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; import org.apache.doris.nereids.trees.plans.logical.LogicalCheckPolicy; -import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; -import org.apache.doris.nereids.trees.plans.logical.LogicalEsScan; import org.apache.doris.nereids.trees.plans.logical.LogicalExcept; -import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; -import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; 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.LogicalIntersect; -import org.apache.doris.nereids.trees.plans.logical.LogicalJdbcScan; import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; import org.apache.doris.nereids.trees.plans.logical.LogicalLimit; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; -import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; -import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalPartitionTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; -import org.apache.doris.nereids.trees.plans.logical.LogicalSchemaScan; import org.apache.doris.nereids.trees.plans.logical.LogicalSelectHint; import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; -import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalTopN; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; import org.apache.doris.nereids.trees.plans.logical.LogicalWindow; @@ -67,31 +54,22 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEConsumer; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEProducer; import org.apache.doris.nereids.trees.plans.physical.PhysicalDistribute; -import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation; -import org.apache.doris.nereids.trees.plans.physical.PhysicalEsScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalExcept; -import org.apache.doris.nereids.trees.plans.physical.PhysicalFileScan; -import org.apache.doris.nereids.trees.plans.physical.PhysicalFileSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; import org.apache.doris.nereids.trees.plans.physical.PhysicalGenerate; 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.PhysicalIntersect; -import org.apache.doris.nereids.trees.plans.physical.PhysicalJdbcScan; 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.PhysicalOlapScan; -import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; -import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalPartitionTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.trees.plans.physical.PhysicalQuickSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalRepeat; -import org.apache.doris.nereids.trees.plans.physical.PhysicalSchemaScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate; -import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow; @@ -102,7 +80,7 @@ * @param Return type of each visit method. * @param Context type. */ -public abstract class PlanVisitor implements CommandVisitor { +public abstract class PlanVisitor implements CommandVisitor, RelationVisitor, SinkVisitor { public abstract R visit(Plan plan, C context); @@ -110,142 +88,119 @@ public abstract class PlanVisitor implements CommandVisitor { // commands // ******************************* + @Override public R visitCommand(Command command, C context) { return visit(command, context); } // ******************************* - // Logical plans + // relations // ******************************* - public R visitLogicalCTE(LogicalCTE cte, C context) { - return visit(cte, context); - } - - public R visitSubQueryAlias(LogicalSubQueryAlias alias, C context) { - return visit(alias, context); - } - - public R visitUnboundOneRowRelation(UnboundOneRowRelation oneRowRelation, C context) { - return visit(oneRowRelation, context); - } - - public R visitUnboundOlapTableSink(UnboundOlapTableSink unboundOlapTableSink, C context) { - return visit(unboundOlapTableSink, context); - } - - public R visitLogicalEmptyRelation(LogicalEmptyRelation emptyRelation, C context) { - return visit(emptyRelation, context); + @Override + public R visitLogicalRelation(LogicalRelation relation, C context) { + return visit(relation, context); } - public R visitLogicalOneRowRelation(LogicalOneRowRelation oneRowRelation, C context) { - return visit(oneRowRelation, context); + @Override + public R visitPhysicalRelation(PhysicalRelation physicalRelation, C context) { + return visit(physicalRelation, context); } - public R visitUnboundRelation(UnboundRelation relation, C context) { - return visit(relation, context); - } + // ******************************* + // sinks + // ******************************* - public R visitUnboundTVFRelation(UnboundTVFRelation unboundTVFRelation, C context) { - return visit(unboundTVFRelation, context); + @Override + public R visitLogicalSink(LogicalSink logicalSink, C context) { + return visit(logicalSink, context); } - public R visitLogicalRelation(LogicalRelation relation, C context) { - return visit(relation, context); + @Override + public R visitPhysicalSink(PhysicalSink physicalSink, C context) { + return visit(physicalSink, context); } - public R visitLogicalSelectHint(LogicalSelectHint hint, C context) { - return visit(hint, context); - } + // ******************************* + // Logical plans + // ******************************* public R visitLogicalAggregate(LogicalAggregate aggregate, C context) { return visit(aggregate, context); } - public R visitLogicalRepeat(LogicalRepeat repeat, C context) { - return visit(repeat, context); + public R visitLogicalApply(LogicalApply apply, C context) { + return visit(apply, context); } - public R visitLogicalFilter(LogicalFilter filter, C context) { - return visit(filter, context); + public R visitLogicalAssertNumRows(LogicalAssertNumRows assertNumRows, C context) { + return visit(assertNumRows, context); } public R visitLogicalCheckPolicy(LogicalCheckPolicy checkPolicy, C context) { return visit(checkPolicy, context); } - public R visitLogicalOlapScan(LogicalOlapScan olapScan, C context) { - return visitLogicalRelation(olapScan, context); - } - - public R visitLogicalSchemaScan(LogicalSchemaScan schemaScan, C context) { - return visitLogicalRelation(schemaScan, context); + public R visitLogicalCTE(LogicalCTE cte, C context) { + return visit(cte, context); } - public R visitLogicalFileScan(LogicalFileScan fileScan, C context) { - return visitLogicalRelation(fileScan, context); + public R visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, C context) { + return visit(cteAnchor, context); } - public R visitLogicalTVFRelation(LogicalTVFRelation tvfRelation, C context) { - return visitLogicalRelation(tvfRelation, context); + public R visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, C context) { + return visit(cteConsumer, context); } - public R visitLogicalJdbcScan(LogicalJdbcScan jdbcScan, C context) { - return visitLogicalRelation(jdbcScan, context); + public R visitLogicalCTEProducer(LogicalCTEProducer cteProducer, C context) { + return visit(cteProducer, context); } - public R visitLogicalEsScan(LogicalEsScan esScan, C context) { - return visitLogicalRelation(esScan, context); + public R visitLogicalFilter(LogicalFilter filter, C context) { + return visit(filter, context); } - public R visitLogicalProject(LogicalProject project, C context) { - return visit(project, context); + public R visitLogicalGenerate(LogicalGenerate generate, C context) { + return visit(generate, context); } - public R visitLogicalSort(LogicalSort sort, C context) { - return visit(sort, context); + public R visitGroupPlan(GroupPlan groupPlan, C context) { + return visit(groupPlan, context); } - public R visitLogicalTopN(LogicalTopN topN, C context) { - return visit(topN, context); + public R visitLogicalHaving(LogicalHaving having, C context) { + return visit(having, context); } - public R visitLogicalPartitionTopN(LogicalPartitionTopN partitionTopN, C context) { - return visit(partitionTopN, context); + public R visitLogicalJoin(LogicalJoin join, C context) { + return visit(join, context); } public R visitLogicalLimit(LogicalLimit limit, C context) { return visit(limit, context); } - public R visitLogicalJoin(LogicalJoin join, C context) { - return visit(join, context); - } - - public R visitGroupPlan(GroupPlan groupPlan, C context) { - return visit(groupPlan, context); + public R visitLogicalPartitionTopN(LogicalPartitionTopN partitionTopN, C context) { + return visit(partitionTopN, context); } - public R visitLogicalApply(LogicalApply apply, C context) { - return visit(apply, context); + public R visitLogicalProject(LogicalProject project, C context) { + return visit(project, context); } - public R visitLogicalAssertNumRows(LogicalAssertNumRows assertNumRows, C context) { - return visit(assertNumRows, context); + public R visitLogicalRepeat(LogicalRepeat repeat, C context) { + return visit(repeat, context); } - public R visitLogicalHaving(LogicalHaving having, C context) { - return visit(having, context); + public R visitLogicalSelectHint(LogicalSelectHint hint, C context) { + return visit(hint, context); } public R visitLogicalSetOperation(LogicalSetOperation setOperation, C context) { return visit(setOperation, context); } - public R visitLogicalUnion(LogicalUnion union, C context) { - return visitLogicalSetOperation(union, context); - } - public R visitLogicalExcept(LogicalExcept except, C context) { return visitLogicalSetOperation(except, context); } @@ -254,16 +209,24 @@ public R visitLogicalIntersect(LogicalIntersect intersect, C context) { return visitLogicalSetOperation(intersect, context); } - public R visitLogicalGenerate(LogicalGenerate generate, C context) { - return visit(generate, context); + public R visitLogicalUnion(LogicalUnion union, C context) { + return visitLogicalSetOperation(union, context); } - public R visitLogicalWindow(LogicalWindow window, C context) { - return visit(window, context); + public R visitLogicalSort(LogicalSort sort, C context) { + return visit(sort, context); } - public R visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, C context) { - return visit(olapTableSink, context); + public R visitLogicalSubQueryAlias(LogicalSubQueryAlias alias, C context) { + return visit(alias, context); + } + + public R visitLogicalTopN(LogicalTopN topN, C context) { + return visit(topN, context); + } + + public R visitLogicalWindow(LogicalWindow window, C context) { + return visit(window, context); } // ******************************* @@ -274,85 +237,33 @@ public R visitPhysicalHashAggregate(PhysicalHashAggregate agg, C return visit(agg, context); } - public R visitPhysicalRepeat(PhysicalRepeat repeat, C context) { - return visit(repeat, context); - } - - public R visitPhysicalScan(PhysicalRelation scan, C context) { - return visit(scan, context); - } - - public R visitPhysicalEmptyRelation(PhysicalEmptyRelation emptyRelation, C context) { - return visit(emptyRelation, context); - } - - public R visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, C context) { - return visit(oneRowRelation, context); - } - - public R visitPhysicalOlapScan(PhysicalOlapScan olapScan, C context) { - return visitPhysicalScan(olapScan, context); - } - - public R visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, C context) { - return visitPhysicalScan(schemaScan, context); - } - - public R visitPhysicalFileScan(PhysicalFileScan fileScan, C context) { - return visitPhysicalScan(fileScan, context); - } - - public R visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, C context) { - return visitPhysicalScan(jdbcScan, context); - } - - public R visitPhysicalEsScan(PhysicalEsScan esScan, C context) { - return visitPhysicalScan(esScan, context); - } - public R visitPhysicalStorageLayerAggregate(PhysicalStorageLayerAggregate storageLayerAggregate, C context) { return storageLayerAggregate.getRelation().accept(this, context); } - public R visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, C context) { - return visitPhysicalScan(tvfRelation, context); - } - - public R visitAbstractPhysicalSort(AbstractPhysicalSort sort, C context) { - return visit(sort, context); - } - - public R visitPhysicalQuickSort(PhysicalQuickSort sort, C context) { - return visitAbstractPhysicalSort(sort, context); - } - - public R visitPhysicalWindow(PhysicalWindow window, C context) { - return visit(window, context); - } - - public R visitPhysicalTopN(PhysicalTopN topN, C context) { - return visitAbstractPhysicalSort(topN, context); + public R visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, C context) { + return visit(assertNumRows, context); } - public R visitPhysicalPartitionTopN(PhysicalPartitionTopN partitionTopN, C context) { - return visit(partitionTopN, context); + public R visitPhysicalCTEAnchor( + PhysicalCTEAnchor cteAnchor, C context) { + return visit(cteAnchor, context); } - public R visitPhysicalLimit(PhysicalLimit limit, C context) { - return visit(limit, context); + public R visitPhysicalCTEConsumer(PhysicalCTEConsumer cteConsumer, C context) { + return visit(cteConsumer, context); } public R visitPhysicalCTEProducer(PhysicalCTEProducer cteProducer, C context) { return visit(cteProducer, context); } - public R visitPhysicalCTEConsumer(PhysicalCTEConsumer cteConsumer, C context) { - return visit(cteConsumer, context); + public R visitPhysicalFilter(PhysicalFilter filter, C context) { + return visit(filter, context); } - public R visitPhysicalCTEAnchor( - PhysicalCTEAnchor cteAnchor, C context) { - return visit(cteAnchor, context); + public R visitPhysicalGenerate(PhysicalGenerate generate, C context) { + return visit(generate, context); } public R visitAbstractPhysicalJoin(AbstractPhysicalJoin join, C context) { @@ -368,22 +279,26 @@ public R visitPhysicalNestedLoopJoin( return visitAbstractPhysicalJoin(nestedLoopJoin, context); } + public R visitPhysicalLimit(PhysicalLimit limit, C context) { + return visit(limit, context); + } + + public R visitPhysicalPartitionTopN(PhysicalPartitionTopN partitionTopN, C context) { + return visit(partitionTopN, context); + } + public R visitPhysicalProject(PhysicalProject project, C context) { return visit(project, context); } - public R visitPhysicalFilter(PhysicalFilter filter, C context) { - return visit(filter, context); + public R visitPhysicalRepeat(PhysicalRepeat repeat, C context) { + return visit(repeat, context); } public R visitPhysicalSetOperation(PhysicalSetOperation setOperation, C context) { return visit(setOperation, context); } - public R visitPhysicalUnion(PhysicalUnion union, C context) { - return visitPhysicalSetOperation(union, context); - } - public R visitPhysicalExcept(PhysicalExcept except, C context) { return visitPhysicalSetOperation(except, context); } @@ -392,43 +307,31 @@ public R visitPhysicalIntersect(PhysicalIntersect intersect, C context) { return visitPhysicalSetOperation(intersect, context); } - public R visitPhysicalGenerate(PhysicalGenerate generate, C context) { - return visit(generate, context); - } - - public R visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, C context) { - return visit(olapTableSink, context); - } - - // ******************************* - // Physical enforcer - // ******************************* - - public R visitPhysicalDistribute(PhysicalDistribute distribute, C context) { - return visit(distribute, context); + public R visitPhysicalUnion(PhysicalUnion union, C context) { + return visitPhysicalSetOperation(union, context); } - public R visitPhysicalAssertNumRows(PhysicalAssertNumRows assertNumRows, C context) { - return visit(assertNumRows, context); + public R visitAbstractPhysicalSort(AbstractPhysicalSort sort, C context) { + return visit(sort, context); } - public R visitLogicalCTEProducer(LogicalCTEProducer cteProducer, C context) { - return visit(cteProducer, context); + public R visitPhysicalQuickSort(PhysicalQuickSort sort, C context) { + return visitAbstractPhysicalSort(sort, context); } - public R visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, C context) { - return visit(cteConsumer, context); + public R visitPhysicalTopN(PhysicalTopN topN, C context) { + return visitAbstractPhysicalSort(topN, context); } - public R visitLogicalCTEAnchor(LogicalCTEAnchor cteAnchor, C context) { - return visit(cteAnchor, context); + public R visitPhysicalWindow(PhysicalWindow window, C context) { + return visit(window, context); } - public R visitLogicalFileSink(LogicalFileSink fileSink, C context) { - return visit(fileSink, context); - } + // ******************************* + // Physical enforcer + // ******************************* - public R visitPhysicalFileSink(PhysicalFileSink fileSink, C context) { - return visit(fileSink, context); + public R visitPhysicalDistribute(PhysicalDistribute distribute, C context) { + return visit(distribute, context); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/RelationVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/RelationVisitor.java new file mode 100644 index 00000000000000..af65f43d9d4b53 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/RelationVisitor.java @@ -0,0 +1,142 @@ +// 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.visitor; + +import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; +import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundTVFRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalEsScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalJdbcScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalSchemaScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalTVFRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalEsScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFileScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalJdbcScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOneRowRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRelation; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSchemaScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalTVFRelation; + +/** + * relation visitor + */ +public interface RelationVisitor { + + // ******************************* + // interface + // ******************************* + + R visitLogicalRelation(LogicalRelation logicalRelation, C context); + + R visitPhysicalRelation(PhysicalRelation physicalRelation, C context); + + // ******************************* + // unbound relations + // ******************************* + + default R visitUnboundOneRowRelation(UnboundOneRowRelation oneRowRelation, C context) { + return visitLogicalRelation(oneRowRelation, context); + } + + default R visitUnboundRelation(UnboundRelation relation, C context) { + return visitLogicalRelation(relation, context); + } + + default R visitUnboundTVFRelation(UnboundTVFRelation unboundTVFRelation, C context) { + return visitLogicalRelation(unboundTVFRelation, context); + } + + // ******************************* + // logical relations + // ******************************* + + default R visitLogicalEmptyRelation(LogicalEmptyRelation emptyRelation, C context) { + return visitLogicalRelation(emptyRelation, context); + } + + default R visitLogicalEsScan(LogicalEsScan esScan, C context) { + return visitLogicalRelation(esScan, context); + } + + default R visitLogicalFileScan(LogicalFileScan fileScan, C context) { + return visitLogicalRelation(fileScan, context); + } + + default R visitLogicalJdbcScan(LogicalJdbcScan jdbcScan, C context) { + return visitLogicalRelation(jdbcScan, context); + } + + default R visitLogicalOlapScan(LogicalOlapScan olapScan, C context) { + return visitLogicalRelation(olapScan, context); + } + + default R visitLogicalOneRowRelation(LogicalOneRowRelation oneRowRelation, C context) { + return visitLogicalRelation(oneRowRelation, context); + } + + default R visitLogicalSchemaScan(LogicalSchemaScan schemaScan, C context) { + return visitLogicalRelation(schemaScan, context); + } + + default R visitLogicalTVFRelation(LogicalTVFRelation tvfRelation, C context) { + return visitLogicalRelation(tvfRelation, context); + } + + // ******************************* + // physical relations + // ******************************* + + default R visitPhysicalEmptyRelation(PhysicalEmptyRelation emptyRelation, C context) { + return visitPhysicalRelation(emptyRelation, context); + } + + default R visitPhysicalEsScan(PhysicalEsScan esScan, C context) { + return visitPhysicalRelation(esScan, context); + } + + default R visitPhysicalFileScan(PhysicalFileScan fileScan, C context) { + return visitPhysicalRelation(fileScan, context); + } + + default R visitPhysicalJdbcScan(PhysicalJdbcScan jdbcScan, C context) { + return visitPhysicalRelation(jdbcScan, context); + } + + default R visitPhysicalOlapScan(PhysicalOlapScan olapScan, C context) { + return visitPhysicalRelation(olapScan, context); + } + + default R visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, C context) { + return visitPhysicalRelation(oneRowRelation, context); + } + + default R visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, C context) { + return visitPhysicalRelation(schemaScan, context); + } + + default R visitPhysicalTVFRelation(PhysicalTVFRelation tvfRelation, C context) { + return visitPhysicalRelation(tvfRelation, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java new file mode 100644 index 00000000000000..629deaacf7d62b --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java @@ -0,0 +1,73 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.visitor; + +import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalFileSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; + +/** + * sink visitor + */ +public interface SinkVisitor { + + // ******************************* + // interface + // ******************************* + + R visitLogicalSink(LogicalSink logicalSink, C context); + + R visitPhysicalSink(PhysicalSink physicalSink, C context); + + // ******************************* + // unbound + // ******************************* + + default R visitUnboundOlapTableSink(UnboundOlapTableSink unboundOlapTableSink, C context) { + return visitLogicalSink(unboundOlapTableSink, context); + } + + // ******************************* + // logical + // ******************************* + + default R visitLogicalFileSink(LogicalFileSink fileSink, C context) { + return visitLogicalSink(fileSink, context); + } + + default R visitLogicalOlapTableSink(LogicalOlapTableSink olapTableSink, C context) { + return visitLogicalSink(olapTableSink, context); + } + + // ******************************* + // physical + // ******************************* + + default R visitPhysicalFileSink(PhysicalFileSink fileSink, C context) { + return visitPhysicalSink(fileSink, context); + } + + default R visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, C context) { + return visitPhysicalSink(olapTableSink, context); + } +} From 6efaee56bc798602a13a0017c96a3d76a23502d4 Mon Sep 17 00:00:00 2001 From: morrySnow <101034200+morrySnow@users.noreply.github.com> Date: Fri, 28 Jul 2023 11:31:09 +0800 Subject: [PATCH 7/8] [refactor](Nereids) add result sink node (#22254) use ResultSink as query root node to let plan of query statement has the same pattern with insert statement --- .../analyzer/UnboundOlapTableSink.java | 3 +- .../nereids/analyzer/UnboundResultSink.java | 89 +++++++++ .../translator/PhysicalPlanTranslator.java | 7 + .../nereids/parser/LogicalPlanBuilder.java | 8 +- .../properties/RequestPropertyDeriver.java | 7 + .../apache/doris/nereids/rules/RuleSet.java | 2 + .../apache/doris/nereids/rules/RuleType.java | 2 + .../rules/analysis/BindExpression.java | 9 + .../nereids/rules/analysis/BindRelation.java | 5 + ...LogicalResultSinkToPhysicalResultSink.java | 43 +++++ .../rules/rewrite/AddDefaultLimit.java | 3 +- .../rules/rewrite/PullUpCteAnchor.java | 8 - .../doris/nereids/trees/plans/PlanType.java | 132 +++++++------ .../plans/logical/LogicalCTEConsumer.java | 6 +- .../plans/logical/LogicalResultSink.java | 122 ++++++++++++ .../plans/physical/PhysicalCTEConsumer.java | 2 +- .../plans/physical/PhysicalCTEProducer.java | 3 +- .../plans/physical/PhysicalDistribute.java | 4 +- .../plans/physical/PhysicalHashAggregate.java | 4 +- .../plans/physical/PhysicalResultSink.java | 125 +++++++++++++ .../trees/plans/visitor/SinkVisitor.java | 15 ++ .../apache/doris/nereids/JoinHintTest.java | 31 ++-- .../doris/nereids/parser/LimitClauseTest.java | 12 +- .../nereids/parser/NereidsParserTest.java | 42 ++--- .../rules/analysis/AnalyzeCTETest.java | 10 +- .../rules/analysis/AnalyzeSubQueryTest.java | 6 +- .../analysis/AnalyzeWhereSubqueryTest.java | 58 +++--- .../rules/analysis/BindRelationTest.java | 2 +- .../analysis/CheckExpressionLegalityTest.java | 4 +- .../analysis/FillUpMissingSlotsTest.java | 44 ++--- .../rules/analysis/FunctionRegistryTest.java | 6 +- .../rules/rewrite/ColumnPruningTest.java | 24 +-- .../rules/rewrite/InferPredicatesTest.java | 44 ++--- ...ushdownExpressionsInHashConditionTest.java | 6 +- .../rewrite/mv/SelectRollupIndexTest.java | 2 + .../doris/nereids/sqltest/InferTest.java | 6 +- .../doris/nereids/sqltest/JoinTest.java | 21 ++- .../nereids/trees/expressions/ViewTest.java | 4 +- .../shape/query1.out | 49 ++--- .../shape/query10.out | 89 ++++----- .../shape/query11.out | 35 ++-- .../shape/query12.out | 47 ++--- .../shape/query13.out | 61 +++--- .../shape/query14.out | 163 +++++++++-------- .../shape/query15.out | 44 ++--- .../shape/query16.out | 59 +++--- .../shape/query17.out | 80 ++++---- .../shape/query18.out | 80 ++++---- .../shape/query19.out | 63 +++---- .../shape/query2.out | 51 +++--- .../shape/query20.out | 43 ++--- .../shape/query21.out | 37 ++-- .../shape/query22.out | 37 ++-- .../shape/query23.out | 81 ++++---- .../shape/query24.out | 41 +++-- .../shape/query25.out | 72 ++++---- .../shape/query26.out | 57 +++--- .../shape/query27.out | 59 +++--- .../shape/query28.out | 113 +++++------- .../shape/query29.out | 75 ++++---- .../shape/query3.out | 37 ++-- .../shape/query30.out | 47 ++--- .../shape/query31.out | 57 +++--- .../shape/query32.out | 41 +++-- .../shape/query33.out | 93 +++++----- .../shape/query34.out | 53 +++--- .../shape/query35.out | 93 +++++----- .../shape/query36.out | 59 +++--- .../shape/query37.out | 45 ++--- .../shape/query38.out | 115 ++++++------ .../shape/query39.out | 27 +-- .../shape/query4.out | 67 +++---- .../shape/query40.out | 38 ++-- .../shape/query41.out | 39 ++-- .../shape/query42.out | 35 ++-- .../shape/query43.out | 35 ++-- .../shape/query44.out | 77 ++++---- .../shape/query45.out | 75 ++++---- .../shape/query46.out | 69 +++---- .../shape/query47.out | 37 ++-- .../shape/query48.out | 51 +++--- .../shape/query49.out | 155 ++++++++-------- .../shape/query5.out | 129 ++++++------- .../shape/query50.out | 49 ++--- .../shape/query51.out | 75 ++++---- .../shape/query52.out | 37 ++-- .../shape/query53.out | 57 +++--- .../shape/query54.out | 149 +++++++-------- .../shape/query55.out | 37 ++-- .../shape/query56.out | 161 ++++++++-------- .../shape/query57.out | 37 ++-- .../shape/query58.out | 123 +++++++------ .../shape/query59.out | 66 +++---- .../shape/query6.out | 66 +++---- .../shape/query60.out | 157 ++++++++-------- .../shape/query61.out | 107 +++++------ .../shape/query62.out | 37 ++-- .../shape/query63.out | 57 +++--- .../shape/query65.out | 60 +++--- .../shape/query66.out | 95 +++++----- .../shape/query67.out | 51 +++--- .../shape/query68.out | 72 ++++---- .../shape/query69.out | 87 ++++----- .../shape/query7.out | 57 +++--- .../shape/query70.out | 85 ++++----- .../shape/query71.out | 81 ++++---- .../shape/query72.out | 62 +++---- .../shape/query73.out | 53 +++--- .../shape/query74.out | 43 ++--- .../shape/query75.out | 23 +-- .../shape/query76.out | 81 ++++---- .../shape/query77.out | 173 +++++++++--------- .../shape/query78.out | 113 ++++++------ .../shape/query79.out | 55 +++--- .../shape/query8.out | 73 ++++---- .../shape/query80.out | 163 +++++++++-------- .../shape/query81.out | 47 ++--- .../shape/query82.out | 45 ++--- .../shape/query83.out | 113 ++++++------ .../shape/query84.out | 55 +++--- .../shape/query85.out | 79 ++++---- .../shape/query86.out | 47 ++--- .../shape/query87.out | 113 ++++++------ .../shape/query88.out | 95 +++++----- .../shape/query89.out | 59 +++--- .../shape/query9.out | 22 +-- .../shape/query90.out | 53 +++--- .../shape/query91.out | 83 ++++----- .../shape/query92.out | 41 +++-- .../shape/query93.out | 31 ++-- .../shape/query94.out | 57 +++--- .../shape/query95.out | 73 ++++---- .../shape/query96.out | 43 ++--- .../shape/query97.out | 57 +++--- .../shape/query98.out | 43 ++--- .../shape/query99.out | 37 ++-- .../nereids_tpch_shape_sf1000_p0/shape/q1.out | 19 +- .../shape/q10.out | 46 ++--- .../shape/q11.out | 65 +++---- .../shape/q12.out | 27 +-- .../shape/q13.out | 35 ++-- .../shape/q14.out | 23 +-- .../shape/q15.out | 53 +++--- .../shape/q16.out | 37 ++-- .../shape/q17.out | 33 ++-- .../shape/q18.out | 43 ++--- .../shape/q19.out | 23 +-- .../nereids_tpch_shape_sf1000_p0/shape/q2.out | 57 +++--- .../shape/q20-rewrite.out | 57 +++--- .../shape/q20.out | 57 +++--- .../shape/q21.out | 45 ++--- .../shape/q22.out | 34 ++-- .../nereids_tpch_shape_sf1000_p0/shape/q3.out | 41 +++-- .../nereids_tpch_shape_sf1000_p0/shape/q4.out | 29 +-- .../nereids_tpch_shape_sf1000_p0/shape/q5.out | 64 +++---- .../nereids_tpch_shape_sf1000_p0/shape/q6.out | 13 +- .../nereids_tpch_shape_sf1000_p0/shape/q7.out | 61 +++--- .../nereids_tpch_shape_sf1000_p0/shape/q8.out | 87 ++++----- .../nereids_tpch_shape_sf1000_p0/shape/q9.out | 67 +++---- .../nereids_tpch_shape_sf500_p0/shape/q1.out | 19 +- .../nereids_tpch_shape_sf500_p0/shape/q10.out | 46 ++--- .../nereids_tpch_shape_sf500_p0/shape/q11.out | 65 +++---- .../nereids_tpch_shape_sf500_p0/shape/q12.out | 27 +-- .../nereids_tpch_shape_sf500_p0/shape/q13.out | 35 ++-- .../nereids_tpch_shape_sf500_p0/shape/q14.out | 23 +-- .../nereids_tpch_shape_sf500_p0/shape/q15.out | 53 +++--- .../nereids_tpch_shape_sf500_p0/shape/q16.out | 37 ++-- .../nereids_tpch_shape_sf500_p0/shape/q17.out | 33 ++-- .../nereids_tpch_shape_sf500_p0/shape/q18.out | 43 ++--- .../nereids_tpch_shape_sf500_p0/shape/q19.out | 23 +-- .../nereids_tpch_shape_sf500_p0/shape/q2.out | 57 +++--- .../nereids_tpch_shape_sf500_p0/shape/q20.out | 57 +++--- .../nereids_tpch_shape_sf500_p0/shape/q21.out | 45 ++--- .../nereids_tpch_shape_sf500_p0/shape/q22.out | 34 ++-- .../nereids_tpch_shape_sf500_p0/shape/q3.out | 41 +++-- .../nereids_tpch_shape_sf500_p0/shape/q4.out | 29 +-- .../nereids_tpch_shape_sf500_p0/shape/q5.out | 64 +++---- .../nereids_tpch_shape_sf500_p0/shape/q6.out | 13 +- .../nereids_tpch_shape_sf500_p0/shape/q7.out | 61 +++--- .../nereids_tpch_shape_sf500_p0/shape/q8.out | 87 ++++----- .../nereids_tpch_shape_sf500_p0/shape/q9.out | 63 +++---- 181 files changed, 5102 insertions(+), 4535 deletions(-) create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalResultSinkToPhysicalResultSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java create mode 100644 fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalResultSink.java diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java index 9bf097f994ccf6..484ab16b509427 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.analyzer; +import org.apache.doris.nereids.exceptions.UnboundException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.UnboundLogicalProperties; @@ -143,6 +144,6 @@ public LogicalProperties computeLogicalProperties() { @Override public List computeOutput() { - return child().getOutput(); + throw new UnboundException("output"); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java new file mode 100644 index 00000000000000..9743d0fe8f5eb4 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundResultSink.java @@ -0,0 +1,89 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.analyzer; + +import org.apache.doris.nereids.exceptions.UnboundException; +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.logical.LogicalSink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; + +import java.util.List; +import java.util.Optional; + +/** + * unbound result sink + */ +public class UnboundResultSink extends LogicalSink implements Unbound, Sink { + + public UnboundResultSink(CHILD_TYPE child) { + super(PlanType.LOGICAL_UNBOUND_RESULT_SINK, child); + } + + public UnboundResultSink(Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_UNBOUND_RESULT_SINK, groupExpression, logicalProperties, child); + } + + @Override + public Plan withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, "UnboundResultSink only accepts one child"); + return new UnboundResultSink<>(groupExpression, Optional.of(getLogicalProperties()), children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitUnboundResultSink(this, context); + } + + @Override + public List getExpressions() { + throw new UnsupportedOperationException(this.getClass().getSimpleName() + " don't support getExpression()"); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new UnboundResultSink<>(groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1, "UnboundResultSink only accepts one child"); + return new UnboundResultSink<>(groupExpression, logicalProperties, children.get(0)); + + } + + @Override + public List computeOutput() { + throw new UnboundException("output"); + } + + @Override + public String toString() { + return Utils.toSqlString("UnboundResultSink[" + id.asInt() + "]"); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/glue/translator/PhysicalPlanTranslator.java index 5c8227ccedf9b3..589919e47cf4e5 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 @@ -113,6 +113,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.trees.plans.physical.PhysicalQuickSort; import org.apache.doris.nereids.trees.plans.physical.PhysicalRepeat; +import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalSchemaScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.physical.PhysicalStorageLayerAggregate; @@ -311,6 +312,12 @@ public PlanFragment visitPhysicalDistribute(PhysicalDistribute d * sink Node, in lexicographical order * ******************************************************************************************** */ + @Override + public PlanFragment visitPhysicalResultSink(PhysicalResultSink physicalResultSink, + PlanTranslatorContext context) { + return physicalResultSink.child().accept(this, context); + } + @Override public PlanFragment visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, PlanTranslatorContext context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java index 696ac16f6f36bc..1f578dd554bb31 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java @@ -120,6 +120,7 @@ import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; import org.apache.doris.nereids.analyzer.UnboundOneRowRelation; import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.analyzer.UnboundSlot; import org.apache.doris.nereids.analyzer.UnboundStar; import org.apache.doris.nereids.analyzer.UnboundTVFRelation; @@ -311,7 +312,12 @@ public LogicalPlan visitSingleStatement(SingleStatementContext ctx) { @Override public LogicalPlan visitStatementDefault(StatementDefaultContext ctx) { LogicalPlan plan = plan(ctx.query()); - return withExplain(withOutFile(plan, ctx.outFileClause()), ctx.explain()); + if (ctx.outFileClause() != null) { + plan = withOutFile(plan, ctx.outFileClause()); + } else { + plan = new UnboundResultSink<>(plan); + } + return withExplain(plan, ctx.explain()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java index 11375064f474fc..9b5603fe86f9a0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/properties/RequestPropertyDeriver.java @@ -34,6 +34,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalLimit; import org.apache.doris.nereids.trees.plans.physical.PhysicalNestedLoopJoin; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnion; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -102,6 +103,12 @@ public Void visitPhysicalOlapTableSink(PhysicalOlapTableSink ola return null; } + @Override + public Void visitPhysicalResultSink(PhysicalResultSink physicalResultSink, PlanContext context) { + addRequestPropertyToChildren(PhysicalProperties.GATHER); + return null; + } + /* ******************************************************************************************** * Other Node, in lexicographical order * ******************************************************************************************** */ diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java index acdf9f790a5103..f3e9bf5c55b8d2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleSet.java @@ -62,6 +62,7 @@ import org.apache.doris.nereids.rules.implementation.LogicalPartitionTopNToPhysicalPartitionTopN; import org.apache.doris.nereids.rules.implementation.LogicalProjectToPhysicalProject; import org.apache.doris.nereids.rules.implementation.LogicalRepeatToPhysicalRepeat; +import org.apache.doris.nereids.rules.implementation.LogicalResultSinkToPhysicalResultSink; import org.apache.doris.nereids.rules.implementation.LogicalSchemaScanToPhysicalSchemaScan; import org.apache.doris.nereids.rules.implementation.LogicalSortToPhysicalQuickSort; import org.apache.doris.nereids.rules.implementation.LogicalTVFRelationToPhysicalTVFRelation; @@ -161,6 +162,7 @@ public class RuleSet { .add(new LogicalGenerateToPhysicalGenerate()) .add(new LogicalOlapTableSinkToPhysicalOlapTableSink()) .add(new LogicalFileSinkToPhysicalFileSink()) + .add(new LogicalResultSinkToPhysicalResultSink()) .build(); public static final List ZIG_ZAG_TREE_JOIN_REORDER = planRuleFactories() diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java index 8027ff7b0249db..1b250384a1c2bd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/RuleType.java @@ -29,6 +29,7 @@ public enum RuleType { // binding rules // **** make sure BINDING_UNBOUND_LOGICAL_PLAN is the lowest priority in the rewrite rules. **** + BINDING_RESULT_SINK(RuleTypeClass.REWRITE), BINDING_NON_LEAF_LOGICAL_PLAN(RuleTypeClass.REWRITE), BINDING_ONE_ROW_RELATION_SLOT(RuleTypeClass.REWRITE), BINDING_RELATION(RuleTypeClass.REWRITE), @@ -296,6 +297,7 @@ public enum RuleType { LOGICAL_JDBC_SCAN_TO_PHYSICAL_JDBC_SCAN_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_ES_SCAN_TO_PHYSICAL_ES_SCAN_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_OLAP_TABLE_SINK_TO_PHYSICAL_OLAP_TABLE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), + LOGICAL_RESULT_SINK_TO_PHYSICAL_RESULT_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_FILE_SINK_TO_PHYSICAL_FILE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_ASSERT_NUM_ROWS_TO_PHYSICAL_ASSERT_NUM_ROWS(RuleTypeClass.IMPLEMENTATION), STORAGE_LAYER_AGGREGATE_WITHOUT_PROJECT(RuleTypeClass.IMPLEMENTATION), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java index 9cb49f1c763040..f10bab4e10975a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindExpression.java @@ -65,6 +65,7 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOneRowRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalRepeat; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; import org.apache.doris.nereids.trees.plans.logical.LogicalSort; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; @@ -555,6 +556,14 @@ protected boolean condition(Rule rule, Plan plan) { checkSameNameSlot(subQueryAlias.child(0).getOutput(), subQueryAlias.getAlias()); return subQueryAlias; }) + ), + RuleType.BINDING_RESULT_SINK.build( + unboundResultSink().then(sink -> { + List outputExprs = sink.child().getOutput().stream() + .map(NamedExpression.class::cast) + .collect(ImmutableList.toImmutableList()); + return new LogicalResultSink<>(outputExprs, sink.child()); + }) ) ).stream().map(ruleCondition).collect(ImmutableList.toImmutableList()); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 66e8a7879e12a3..2a68fcd215d00a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -31,6 +31,7 @@ import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.analyzer.Unbound; import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.parser.NereidsParser; import org.apache.doris.nereids.pattern.MatchingContext; @@ -246,6 +247,10 @@ private Plan parseAndAnalyzeHiveView(TableIf table, CascadesContext cascadesCont private Plan parseAndAnalyzeView(String viewSql, CascadesContext parentContext) { LogicalPlan parsedViewPlan = new NereidsParser().parseSingle(viewSql); + // TODO: use a good to do this, such as eliminate UnboundResultSink + if (parsedViewPlan instanceof UnboundResultSink) { + parsedViewPlan = (LogicalPlan) ((UnboundResultSink) parsedViewPlan).child(); + } CascadesContext viewContext = CascadesContext.initContext( parentContext.getStatementContext(), parsedViewPlan, PhysicalProperties.ANY); viewContext.newAnalyzer().analyze(); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalResultSinkToPhysicalResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalResultSinkToPhysicalResultSink.java new file mode 100644 index 00000000000000..47bbad63b38620 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalResultSinkToPhysicalResultSink.java @@ -0,0 +1,43 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.rules.implementation; + +import org.apache.doris.nereids.rules.Rule; +import org.apache.doris.nereids.rules.RuleType; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; + +import java.util.Optional; + +/** + * implement result sink. + */ +public class LogicalResultSinkToPhysicalResultSink extends OneImplementationRuleFactory { + @Override + public Rule build() { + return logicalResultSink().thenApply(ctx -> { + LogicalResultSink sink = ctx.root; + return new PhysicalResultSink<>( + sink.getOutputExprs(), + Optional.empty(), + sink.getLogicalProperties(), + sink.child()); + }).toRule(RuleType.LOGICAL_RESULT_SINK_TO_PHYSICAL_RESULT_SINK_RULE); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java index 93cbf2f7940489..7887836475c821 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/AddDefaultLimit.java @@ -63,8 +63,7 @@ public Plan visitLogicalCTEAnchor(LogicalCTEAnchor logicalSink, StatementContext context) { - Plan child = logicalSink.child().accept(this, context); - return logicalSink.withChildren(child); + return super.visit(logicalSink, context); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java index 356333e4f69907..c6461be27186fc 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/PullUpCteAnchor.java @@ -21,7 +21,6 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEAnchor; import org.apache.doris.nereids.trees.plans.logical.LogicalCTEProducer; -import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.visitor.CustomRewriter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanRewriter; @@ -72,11 +71,4 @@ public LogicalCTEProducer visitLogicalCTEProducer(LogicalCTEProducer logicalSink, List> producers) { - return logicalSink.withChildren(rewriteRoot(logicalSink.child(), producers)); - } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index 2c3a68b093c74e..da98e0e4d14f4e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -21,94 +21,104 @@ * Types for all Plan in Nereids. */ public enum PlanType { + // special + GROUP_PLAN, UNKNOWN, - // logical plan - LOGICAL_OLAP_TABLE_SINK, - LOGICAL_CTE, - LOGICAL_WINDOW, - LOGICAL_SUBQUERY_ALIAS, - LOGICAL_UNBOUND_ONE_ROW_RELATION, + // logical plans + // logical relations + LOGICAL_BOUND_RELATION, + LOGICAL_CTE_CONSUMER, + LOGICAL_FILE_SCAN, LOGICAL_EMPTY_RELATION, + LOGICAL_ES_SCAN, + LOGICAL_JDBC_SCAN, + LOGICAL_OLAP_SCAN, LOGICAL_ONE_ROW_RELATION, + LOGICAL_SCHEMA_SCAN, + LOGICAL_TVF_RELATION, + LOGICAL_UNBOUND_ONE_ROW_RELATION, LOGICAL_UNBOUND_RELATION, LOGICAL_UNBOUND_TVF_RELATION, - LOGICAL_BOUND_RELATION, + + // logical sinks + LOGICAL_FILE_SINK, + LOGICAL_OLAP_TABLE_SINK, + LOGICAL_RESULT_SINK, LOGICAL_UNBOUND_OLAP_TABLE_SINK, - LOGICAL_TVF_RELATION, - LOGICAL_PROJECT, + LOGICAL_UNBOUND_RESULT_SINK, + + // logical others + LOGICAL_AGGREGATE, + LOGICAL_APPLY, + LOGICAL_ASSERT_NUM_ROWS, + LOGICAL_CHECK_POLICY, + LOGICAL_CTE, + LOGICAL_CTE_ANCHOR, + LOGICAL_CTE_PRODUCER, + LOGICAL_EXCEPT, LOGICAL_FILTER, LOGICAL_GENERATE, + LOGICAL_HAVING, + LOGICAL_INTERSECT, LOGICAL_JOIN, - LOGICAL_AGGREGATE, + LOGICAL_LIMIT, + LOGICAL_MULTI_JOIN, + LOGICAL_PARTITION_TOP_N, + LOGICAL_PROJECT, LOGICAL_REPEAT, + LOGICAL_SELECT_HINT, + LOGICAL_SUBQUERY_ALIAS, LOGICAL_SORT, LOGICAL_TOP_N, - LOGICAL_PARTITION_TOP_N, - LOGICAL_LIMIT, - LOGICAL_OLAP_SCAN, - LOGICAL_SCHEMA_SCAN, - LOGICAL_FILE_SCAN, - LOGICAL_JDBC_SCAN, - LOGICAL_ES_SCAN, - LOGICAL_APPLY, - LOGICAL_SELECT_HINT, - LOGICAL_ASSERT_NUM_ROWS, - LOGICAL_HAVING, - LOGICAL_MULTI_JOIN, - LOGICAL_CHECK_POLICY, LOGICAL_UNION, - LOGICAL_EXCEPT, - LOGICAL_INTERSECT, LOGICAL_USING_JOIN, - LOGICAL_CTE_RELATION, - LOGICAL_CTE_ANCHOR, - LOGICAL_CTE_PRODUCER, - LOGICAL_CTE_CONSUMER, - LOGICAL_FILE_SINK, - - GROUP_PLAN, + LOGICAL_WINDOW, - // physical plan - PHYSICAL_OLAP_TABLE_SINK, - PHYSICAL_CTE_PRODUCE, - PHYSICAL_CTE_CONSUME, - PHYSICAL_CTE_ANCHOR, - PHYSICAL_WINDOW, + // physical plans + // logical relations + PHYSICAL_CTE_CONSUMER, PHYSICAL_EMPTY_RELATION, - PHYSICAL_ONE_ROW_RELATION, - PHYSICAL_OLAP_SCAN, + PHYSICAL_ES_SCAN, PHYSICAL_FILE_SCAN, PHYSICAL_JDBC_SCAN, - PHYSICAL_ES_SCAN, - PHYSICAL_TVF_RELATION, + PHYSICAL_ONE_ROW_RELATION, + PHYSICAL_OLAP_SCAN, PHYSICAL_SCHEMA_SCAN, - PHYSICAL_PROJECT, + PHYSICAL_TVF_RELATION, + + // logical sinks + PHYSICAL_FILE_SINK, + PHYSICAL_OLAP_TABLE_SINK, + PHYSICAL_RESULT_SINK, + + // logical others + PHYSICAL_HASH_AGGREGATE, + PHYSICAL_ASSERT_NUM_ROWS, + PHYSICAL_CTE_PRODUCER, + PHYSICAL_CTE_ANCHOR, + PHYSICAL_DISTRIBUTE, + PHYSICAL_EXCEPT, PHYSICAL_FILTER, PHYSICAL_GENERATE, - PHYSICAL_BROADCAST_HASH_JOIN, - PHYSICAL_AGGREGATE, + PHYSICAL_INTERSECT, + PHYSICAL_HASH_JOIN, + PHYSICAL_NESTED_LOOP_JOIN, + PHYSICAL_LIMIT, + PHYSICAL_PARTITION_TOP_N, + PHYSICAL_PROJECT, PHYSICAL_REPEAT, + PHYSICAL_LOCAL_QUICK_SORT, PHYSICAL_QUICK_SORT, PHYSICAL_TOP_N, - PHYSICAL_PARTITION_TOP_N, - PHYSICAL_LOCAL_QUICK_SORT, - PHYSICAL_LIMIT, - PHYSICAL_HASH_JOIN, - PHYSICAL_NESTED_LOOP_JOIN, - PHYSICAL_EXCHANGE, - PHYSICAL_DISTRIBUTION, - PHYSICAL_ASSERT_NUM_ROWS, PHYSICAL_UNION, - PHYSICAL_EXCEPT, - PHYSICAL_INTERSECT, - PHYSICAL_FILE_SINK, + PHYSICAL_WINDOW, - COMMAND, - EXPLAIN_COMMAND, + // commands CREATE_POLICY_COMMAND, - INSERT_INTO_TABLE_COMMAND, - UPDATE_COMMAND, DELETE_COMMAND, - SELECT_INTO_OUTFILE_COMMAND + EXPLAIN_COMMAND, + INSERT_INTO_TABLE_COMMAND, + SELECT_INTO_OUTFILE_COMMAND, + UPDATE_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java index 86f58ce4c1d271..c96d71e5daa0b2 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalCTEConsumer.java @@ -52,7 +52,7 @@ public class LogicalCTEConsumer extends LogicalRelation { */ public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, Map consumerToProducerOutputMap, Map producerToConsumerOutputMap) { - super(relationId, PlanType.LOGICAL_CTE_RELATION, Optional.empty(), Optional.empty()); + super(relationId, PlanType.LOGICAL_CTE_CONSUMER, Optional.empty(), Optional.empty()); this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); this.name = Objects.requireNonNull(name, "name should not null"); this.consumerToProducerOutputMap = Objects.requireNonNull(consumerToProducerOutputMap, @@ -65,7 +65,7 @@ public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, * Logical CTE consumer. */ public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, LogicalPlan producerPlan) { - super(relationId, PlanType.LOGICAL_CTE_RELATION, Optional.empty(), Optional.empty()); + super(relationId, PlanType.LOGICAL_CTE_CONSUMER, Optional.empty(), Optional.empty()); this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); this.name = Objects.requireNonNull(name, "name should not null"); this.consumerToProducerOutputMap = new LinkedHashMap<>(); @@ -79,7 +79,7 @@ public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, Logic public LogicalCTEConsumer(RelationId relationId, CTEId cteId, String name, Map consumerToProducerOutputMap, Map producerToConsumerOutputMap, Optional groupExpression, Optional logicalProperties) { - super(relationId, PlanType.LOGICAL_CTE_RELATION, groupExpression, logicalProperties); + super(relationId, PlanType.LOGICAL_CTE_CONSUMER, groupExpression, logicalProperties); this.cteId = Objects.requireNonNull(cteId, "cteId should not null"); this.name = Objects.requireNonNull(name, "name should not null"); this.consumerToProducerOutputMap = Objects.requireNonNull(consumerToProducerOutputMap, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java new file mode 100644 index 00000000000000..c21422e858a8c2 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalResultSink.java @@ -0,0 +1,122 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.logical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * result sink + */ +public class LogicalResultSink extends LogicalSink implements Sink { + + private final List outputExprs; + + public LogicalResultSink(List outputExprs, CHILD_TYPE child) { + super(PlanType.LOGICAL_RESULT_SINK, child); + this.outputExprs = outputExprs; + } + + public LogicalResultSink(List outputExprs, + Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + super(PlanType.LOGICAL_RESULT_SINK, groupExpression, logicalProperties, child); + this.outputExprs = outputExprs; + } + + public List getOutputExprs() { + return outputExprs; + } + + @Override + public Plan withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "LogicalResultSink's children size must be 1, but real is %s", children.size()); + return new LogicalResultSink<>(outputExprs, children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalResultSink(this, context); + } + + @Override + public List getExpressions() { + return outputExprs; + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new LogicalResultSink<>(outputExprs, groupExpression, Optional.of(getLogicalProperties()), child()); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1, "UnboundResultSink only accepts one child"); + return new LogicalResultSink<>(outputExprs, groupExpression, logicalProperties, children.get(0)); + } + + @Override + public List computeOutput() { + return outputExprs.stream() + .map(NamedExpression::toSlot) + .collect(ImmutableList.toImmutableList()); + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalResultSink[" + id.asInt() + "]", + "outputExprs", outputExprs); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + LogicalResultSink that = (LogicalResultSink) o; + return Objects.equals(outputExprs, that.outputExprs); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), outputExprs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java index e6450a94e8e0ff..260b93e89f7370 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEConsumer.java @@ -71,7 +71,7 @@ public PhysicalCTEConsumer(RelationId relationId, CTEId cteId, public PhysicalCTEConsumer(RelationId relationId, CTEId cteId, Map consumerToProducerSlotMap, Map producerToConsumerSlotMap, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics) { - super(relationId, PlanType.PHYSICAL_CTE_CONSUME, groupExpression, + super(relationId, PlanType.PHYSICAL_CTE_CONSUMER, groupExpression, logicalProperties, physicalProperties, statistics); this.cteId = cteId; this.consumerToProducerSlotMap = ImmutableMap.copyOf(Objects.requireNonNull( diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java index 385aa9e1d33db1..a3917dafc9551b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java @@ -54,7 +54,8 @@ public PhysicalCTEProducer(CTEId cteId, Optional groupExpressio public PhysicalCTEProducer(CTEId cteId, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { - super(PlanType.PHYSICAL_CTE_PRODUCE, groupExpression, logicalProperties, physicalProperties, statistics, child); + super(PlanType.PHYSICAL_CTE_PRODUCER, groupExpression, + logicalProperties, physicalProperties, statistics, child); this.cteId = cteId; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalDistribute.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalDistribute.java index a0dd2401c0bdbc..05535ab55a28ec 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalDistribute.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalDistribute.java @@ -49,14 +49,14 @@ public PhysicalDistribute(DistributionSpec spec, LogicalProperties logicalProper public PhysicalDistribute(DistributionSpec spec, Optional groupExpression, LogicalProperties logicalProperties, CHILD_TYPE child) { - super(PlanType.PHYSICAL_DISTRIBUTION, groupExpression, logicalProperties, child); + super(PlanType.PHYSICAL_DISTRIBUTE, groupExpression, logicalProperties, child); this.distributionSpec = spec; } public PhysicalDistribute(DistributionSpec spec, Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { - super(PlanType.PHYSICAL_DISTRIBUTION, groupExpression, logicalProperties, physicalProperties, statistics, + super(PlanType.PHYSICAL_DISTRIBUTE, groupExpression, logicalProperties, physicalProperties, statistics, child); this.distributionSpec = spec; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java index 620dd88716cc8b..17ad6516fa776a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalHashAggregate.java @@ -88,7 +88,7 @@ public PhysicalHashAggregate(List groupByExpressions, List> partitionExpressions, AggregateParam aggregateParam, boolean maybeUsingStream, Optional groupExpression, LogicalProperties logicalProperties, RequireProperties requireProperties, CHILD_TYPE child) { - super(PlanType.PHYSICAL_AGGREGATE, groupExpression, logicalProperties, child); + super(PlanType.PHYSICAL_HASH_AGGREGATE, groupExpression, logicalProperties, child); this.groupByExpressions = ImmutableList.copyOf( Objects.requireNonNull(groupByExpressions, "groupByExpressions cannot be null")); this.outputExpressions = ImmutableList.copyOf( @@ -114,7 +114,7 @@ public PhysicalHashAggregate(List groupByExpressions, List groupExpression, LogicalProperties logicalProperties, RequireProperties requireProperties, PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { - super(PlanType.PHYSICAL_AGGREGATE, groupExpression, logicalProperties, physicalProperties, statistics, + super(PlanType.PHYSICAL_HASH_AGGREGATE, groupExpression, logicalProperties, physicalProperties, statistics, child); this.groupByExpressions = ImmutableList.copyOf( Objects.requireNonNull(groupByExpressions, "groupByExpressions cannot be null")); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalResultSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalResultSink.java new file mode 100644 index 00000000000000..f3cbeae9c88ab8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalResultSink.java @@ -0,0 +1,125 @@ +// Licensed to the Apache Software Foundation (ASF) under one +// or more contributor license agreements. See the NOTICE file +// distributed with this work for additional information +// regarding copyright ownership. The ASF licenses this file +// to you under the Apache License, Version 2.0 (the +// "License"); you may not use this file except in compliance +// with the License. You may obtain a copy of the License at +// +// http://www.apache.org/licenses/LICENSE-2.0 +// +// Unless required by applicable law or agreed to in writing, +// software distributed under the License is distributed on an +// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY +// KIND, either express or implied. See the License for the +// specific language governing permissions and limitations +// under the License. + +package org.apache.doris.nereids.trees.plans.physical; + +import org.apache.doris.nereids.memo.GroupExpression; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.algebra.Sink; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; +import org.apache.doris.statistics.Statistics; + +import com.google.common.base.Preconditions; +import org.jetbrains.annotations.Nullable; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * result sink + */ +public class PhysicalResultSink extends PhysicalSink implements Sink { + + private final List outputExprs; + + public PhysicalResultSink(List outputExprs, LogicalProperties logicalProperties, + CHILD_TYPE child) { + super(PlanType.PHYSICAL_RESULT_SINK, logicalProperties, child); + this.outputExprs = outputExprs; + } + + public PhysicalResultSink(List outputExprs, Optional groupExpression, + LogicalProperties logicalProperties, CHILD_TYPE child) { + super(PlanType.PHYSICAL_RESULT_SINK, groupExpression, logicalProperties, child); + this.outputExprs = outputExprs; + } + + public PhysicalResultSink(List outputExprs, Optional groupExpression, + LogicalProperties logicalProperties, @Nullable PhysicalProperties physicalProperties, + Statistics statistics, CHILD_TYPE child) { + super(PlanType.PHYSICAL_RESULT_SINK, groupExpression, logicalProperties, physicalProperties, statistics, child); + this.outputExprs = outputExprs; + } + + @Override + public PhysicalResultSink withChildren(List children) { + Preconditions.checkArgument(children.size() == 1, + "PhysicalResultSink's children size must be 1, but real is %s", children.size()); + return new PhysicalResultSink<>(outputExprs, groupExpression, getLogicalProperties(), children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPhysicalResultSink(this, context); + } + + @Override + public List getExpressions() { + return outputExprs; + } + + @Override + public PhysicalResultSink withGroupExpression(Optional groupExpression) { + return new PhysicalResultSink<>(outputExprs, groupExpression, getLogicalProperties(), child()); + } + + @Override + public PhysicalResultSink withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + return new PhysicalResultSink<>(outputExprs, groupExpression, logicalProperties.get(), child()); + } + + @Override + public PhysicalResultSink withPhysicalPropertiesAndStats( + PhysicalProperties physicalProperties, Statistics statistics) { + return new PhysicalResultSink<>(outputExprs, groupExpression, + getLogicalProperties(), physicalProperties, statistics, child()); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + PhysicalResultSink that = (PhysicalResultSink) o; + return Objects.equals(outputExprs, that.outputExprs); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), outputExprs); + } + + @Override + public String toString() { + return Utils.toSqlString("PhysicalResultSink[" + id.asInt() + "]", + "outputExprs", outputExprs); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java index 629deaacf7d62b..63f371aecba4b4 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/SinkVisitor.java @@ -18,12 +18,15 @@ package org.apache.doris.nereids.trees.plans.visitor; import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; +import org.apache.doris.nereids.analyzer.UnboundResultSink; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.logical.LogicalFileSink; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.logical.LogicalResultSink; import org.apache.doris.nereids.trees.plans.logical.LogicalSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalFileSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; +import org.apache.doris.nereids.trees.plans.physical.PhysicalResultSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalSink; /** @@ -47,6 +50,10 @@ default R visitUnboundOlapTableSink(UnboundOlapTableSink unbound return visitLogicalSink(unboundOlapTableSink, context); } + default R visitUnboundResultSink(UnboundResultSink unboundResultSink, C context) { + return visitLogicalSink(unboundResultSink, context); + } + // ******************************* // logical // ******************************* @@ -59,6 +66,10 @@ default R visitLogicalOlapTableSink(LogicalOlapTableSink olapTab return visitLogicalSink(olapTableSink, context); } + default R visitLogicalResultSink(LogicalResultSink logicalResultSink, C context) { + return visitLogicalSink(logicalResultSink, context); + } + // ******************************* // physical // ******************************* @@ -70,4 +81,8 @@ default R visitPhysicalFileSink(PhysicalFileSink fileSink, C con default R visitPhysicalOlapTableSink(PhysicalOlapTableSink olapTableSink, C context) { return visitPhysicalSink(olapTableSink, context); } + + default R visitPhysicalResultSink(PhysicalResultSink physicalResultSink, C context) { + return visitPhysicalSink(physicalResultSink, context); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java index 631a89b7762402..ac43c5698ad8ed 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/JoinHintTest.java @@ -114,22 +114,23 @@ public void testHintWithReorderCrossJoin() throws Exception { PlanChecker.from(connectContext).checkExplain(sql, planner -> { Plan plan = planner.getOptimizedPlan(); MatchingUtils.assertMatches(plan, - physicalDistribute( - physicalProject( - physicalHashJoin( - physicalHashJoin(physicalDistribute().when(dis -> { - DistributionSpec spec = dis.getDistributionSpec(); - Assertions.assertTrue(spec instanceof DistributionSpecHash); - DistributionSpecHash hashSpec = (DistributionSpecHash) spec; - Assertions.assertEquals(ShuffleType.EXECUTION_BUCKETED, - hashSpec.getShuffleType()); - return true; - }), physicalDistribute()), - physicalDistribute() - ).when(join -> join.getHint() == JoinHint.SHUFFLE_RIGHT) + physicalResultSink( + physicalDistribute( + physicalProject( + physicalHashJoin( + physicalHashJoin(physicalDistribute().when(dis -> { + DistributionSpec spec = dis.getDistributionSpec(); + Assertions.assertTrue(spec instanceof DistributionSpecHash); + DistributionSpecHash hashSpec = (DistributionSpecHash) spec; + Assertions.assertEquals(ShuffleType.EXECUTION_BUCKETED, + hashSpec.getShuffleType()); + return true; + }), physicalDistribute()), + physicalDistribute() + ).when(join -> join.getHint() == JoinHint.SHUFFLE_RIGHT) + ) ) - ) - ); + )); }); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/LimitClauseTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/LimitClauseTest.java index 75636ef5982de9..38bbb4c45ebaa6 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/LimitClauseTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/LimitClauseTest.java @@ -23,24 +23,24 @@ public class LimitClauseTest extends ParserTestBase { @Test public void testLimit() { parsePlan("SELECT b FROM test order by a limit 3 offset 100") - .matchesFromRoot( + .matches( logicalLimit( logicalSort() ).when(limit -> limit.getLimit() == 3 && limit.getOffset() == 100) ); parsePlan("SELECT b FROM test order by a limit 100, 3") - .matchesFromRoot( + .matches( logicalLimit( logicalSort() ).when(limit -> limit.getLimit() == 3 && limit.getOffset() == 100) ); parsePlan("SELECT b FROM test limit 3") - .matchesFromRoot(logicalLimit().when(limit -> limit.getLimit() == 3 && limit.getOffset() == 0)); + .matches(logicalLimit().when(limit -> limit.getLimit() == 3 && limit.getOffset() == 0)); parsePlan("SELECT b FROM test order by a limit 3") - .matchesFromRoot( + .matches( logicalLimit( logicalSort() ).when(limit -> limit.getLimit() == 3 && limit.getOffset() == 0) @@ -49,13 +49,13 @@ public void testLimit() { @Test public void testNoLimit() { - parsePlan("select a from tbl order by x").matchesFromRoot(logicalSort()); + parsePlan("select a from tbl order by x").matches(logicalSort()); } @Test public void testNoQueryOrganization() { parsePlan("select a from tbl") - .matchesFromRoot( + .matches( logicalProject( logicalCheckPolicy( unboundRelation() diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java index 63da2752fb118a..fd4d1f2e1287f1 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java @@ -80,7 +80,7 @@ public void testErrorListener() { @Test public void testPostProcessor() { parsePlan("select `AD``D` from t1 where a = 1") - .matchesFromRoot( + .matches( logicalProject().when(p -> "AD`D".equals(p.getProjects().get(0).getName())) ); } @@ -90,17 +90,17 @@ public void testParseCTE() { NereidsParser nereidsParser = new NereidsParser(); LogicalPlan logicalPlan; String cteSql1 = "with t1 as (select s_suppkey from supplier where s_suppkey < 10) select * from t1"; - logicalPlan = nereidsParser.parseSingle(cteSql1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(cteSql1).child(0); Assertions.assertEquals(PlanType.LOGICAL_CTE, logicalPlan.getType()); Assertions.assertEquals(((LogicalCTE) logicalPlan).getAliasQueries().size(), 1); String cteSql2 = "with t1 as (select s_suppkey from supplier), t2 as (select s_suppkey from t1) select * from t2"; - logicalPlan = nereidsParser.parseSingle(cteSql2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(cteSql2).child(0); Assertions.assertEquals(PlanType.LOGICAL_CTE, logicalPlan.getType()); Assertions.assertEquals(((LogicalCTE) logicalPlan).getAliasQueries().size(), 2); String cteSql3 = "with t1 (key, name) as (select s_suppkey, s_name from supplier) select * from t1"; - logicalPlan = nereidsParser.parseSingle(cteSql3); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(cteSql3).child(0); Assertions.assertEquals(PlanType.LOGICAL_CTE, logicalPlan.getType()); Assertions.assertEquals(((LogicalCTE) logicalPlan).getAliasQueries().size(), 1); Optional> columnAliases = ((LogicalCTE) logicalPlan).getAliasQueries().get(0).getColumnAliases(); @@ -112,12 +112,12 @@ public void testParseWindowFunctions() { NereidsParser nereidsParser = new NereidsParser(); LogicalPlan logicalPlan; String windowSql1 = "select k1, rank() over(partition by k1 order by k1) as ranking from t1"; - logicalPlan = nereidsParser.parseSingle(windowSql1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(windowSql1).child(0); Assertions.assertEquals(PlanType.LOGICAL_PROJECT, logicalPlan.getType()); Assertions.assertEquals(((LogicalProject) logicalPlan).getProjects().size(), 2); String windowSql2 = "select k1, sum(k2), rank() over(partition by k1 order by k1) as ranking from t1 group by k1"; - logicalPlan = nereidsParser.parseSingle(windowSql2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(windowSql2).child(0); Assertions.assertEquals(PlanType.LOGICAL_AGGREGATE, logicalPlan.getType()); Assertions.assertEquals(((LogicalAggregate) logicalPlan).getOutputExpressions().size(), 3); @@ -135,7 +135,7 @@ public void testExplainNormal() { ExplainCommand explainCommand = (ExplainCommand) logicalPlan; ExplainLevel explainLevel = explainCommand.getLevel(); Assertions.assertEquals(ExplainLevel.NORMAL, explainLevel); - logicalPlan = explainCommand.getLogicalPlan(); + logicalPlan = (LogicalPlan) explainCommand.getLogicalPlan().child(0); LogicalProject logicalProject = (LogicalProject) logicalPlan; Assertions.assertEquals("AD`D", logicalProject.getProjects().get(0).getName()); } @@ -168,7 +168,7 @@ public void testParseSQL() { Assertions.assertEquals(2, statementBases.size()); Assertions.assertTrue(statementBases.get(0) instanceof LogicalPlanAdapter); Assertions.assertTrue(statementBases.get(1) instanceof LogicalPlanAdapter); - LogicalPlan logicalPlan0 = ((LogicalPlanAdapter) statementBases.get(0)).getLogicalPlan(); + LogicalPlan logicalPlan0 = (LogicalPlan) ((LogicalPlanAdapter) statementBases.get(0)).getLogicalPlan().child(0); LogicalPlan logicalPlan1 = ((LogicalPlanAdapter) statementBases.get(1)).getLogicalPlan(); Assertions.assertTrue(logicalPlan0 instanceof LogicalProject); Assertions.assertTrue(logicalPlan1 instanceof ExplainCommand); @@ -181,57 +181,57 @@ public void testParseJoin() { LogicalJoin logicalJoin; String innerJoin1 = "SELECT t1.a FROM t1 INNER JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(innerJoin1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(innerJoin1).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.INNER_JOIN, logicalJoin.getJoinType()); String innerJoin2 = "SELECT t1.a FROM t1 JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(innerJoin2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(innerJoin2).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.INNER_JOIN, logicalJoin.getJoinType()); String leftJoin1 = "SELECT t1.a FROM t1 LEFT JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(leftJoin1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(leftJoin1).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.LEFT_OUTER_JOIN, logicalJoin.getJoinType()); String leftJoin2 = "SELECT t1.a FROM t1 LEFT OUTER JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(leftJoin2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(leftJoin2).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.LEFT_OUTER_JOIN, logicalJoin.getJoinType()); String rightJoin1 = "SELECT t1.a FROM t1 RIGHT JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(rightJoin1); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(rightJoin1).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.RIGHT_OUTER_JOIN, logicalJoin.getJoinType()); String rightJoin2 = "SELECT t1.a FROM t1 RIGHT OUTER JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(rightJoin2); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(rightJoin2).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.RIGHT_OUTER_JOIN, logicalJoin.getJoinType()); String leftSemiJoin = "SELECT t1.a FROM t1 LEFT SEMI JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(leftSemiJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(leftSemiJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.LEFT_SEMI_JOIN, logicalJoin.getJoinType()); String rightSemiJoin = "SELECT t2.a FROM t1 RIGHT SEMI JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(rightSemiJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(rightSemiJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.RIGHT_SEMI_JOIN, logicalJoin.getJoinType()); String leftAntiJoin = "SELECT t1.a FROM t1 LEFT ANTI JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(leftAntiJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(leftAntiJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.LEFT_ANTI_JOIN, logicalJoin.getJoinType()); String righAntiJoin = "SELECT t2.a FROM t1 RIGHT ANTI JOIN t2 ON t1.id = t2.id;"; - logicalPlan = nereidsParser.parseSingle(righAntiJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(righAntiJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.RIGHT_ANTI_JOIN, logicalJoin.getJoinType()); String crossJoin = "SELECT t1.a FROM t1 CROSS JOIN t2;"; - logicalPlan = nereidsParser.parseSingle(crossJoin); + logicalPlan = (LogicalPlan) nereidsParser.parseSingle(crossJoin).child(0); logicalJoin = (LogicalJoin) logicalPlan.child(0); Assertions.assertEquals(JoinType.CROSS_JOIN, logicalJoin.getJoinType()); } @@ -252,7 +252,7 @@ void parseJoinEmptyConditionError() { public void testParseDecimal() { String f1 = "SELECT col1 * 0.267081789095306 FROM t"; NereidsParser nereidsParser = new NereidsParser(); - LogicalPlan logicalPlan = nereidsParser.parseSingle(f1); + LogicalPlan logicalPlan = (LogicalPlan) nereidsParser.parseSingle(f1).child(0); long doubleCount = logicalPlan .getExpressions() .stream() @@ -334,7 +334,7 @@ public void testJoinHint() { public void testParseCast() { String sql = "SELECT CAST(1 AS DECIMAL(20, 6)) FROM t"; NereidsParser nereidsParser = new NereidsParser(); - LogicalPlan logicalPlan = nereidsParser.parseSingle(sql); + LogicalPlan logicalPlan = (LogicalPlan) nereidsParser.parseSingle(sql).child(0); Cast cast = (Cast) logicalPlan.getExpressions().get(0).child(0); if (Config.enable_decimal_conversion) { DecimalV3Type decimalV3Type = (DecimalV3Type) cast.getDataType(); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java index 7e95409118c256..522f198e3ff774 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeCTETest.java @@ -153,7 +153,7 @@ public void testCTEInHavingAndSubquery() { public void testCTEWithAlias() { PlanChecker.from(connectContext) .analyze(cteConsumerJoin) - .matchesFromRoot( + .matches( logicalCTEAnchor( logicalCTEProducer(), logicalCTEAnchor( @@ -173,7 +173,7 @@ public void testCTEWithAlias() { public void testCTEWithAnExistedTableOrViewName() { PlanChecker.from(connectContext) .analyze(cteReferToAnotherOne) - .matchesFromRoot( + .matches( logicalCTEAnchor( logicalCTEProducer(), logicalCTEAnchor( @@ -191,7 +191,7 @@ public void testCTEWithAnExistedTableOrViewName() { public void testDifferenceRelationId() { PlanChecker.from(connectContext) .analyze(cteWithDiffRelationId) - .matchesFromRoot( + .matches( logicalCTEAnchor( logicalCTEProducer(), logicalProject( @@ -212,7 +212,7 @@ public void testDifferenceRelationId() { public void testCteInTheMiddle() { PlanChecker.from(connectContext) .analyze(cteInTheMiddle) - .matchesFromRoot( + .matches( logicalProject( logicalSubQueryAlias( logicalCTEAnchor( @@ -231,7 +231,7 @@ public void testCteInTheMiddle() { public void testCteNested() { PlanChecker.from(connectContext) .analyze(cteNested) - .matchesFromRoot( + .matches( logicalCTEAnchor( logicalCTEProducer( logicalSubQueryAlias( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeSubQueryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeSubQueryTest.java index e73713bc7dfb08..26496a4dfeaea3 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeSubQueryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeSubQueryTest.java @@ -105,7 +105,7 @@ public void testCaseSubQuery() { PlanChecker.from(connectContext) .analyze(testSql.get(0)) .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) - .matchesFromRoot( + .matches( logicalProject( logicalProject( logicalProject( @@ -129,7 +129,7 @@ public void testCaseMixed() { PlanChecker.from(connectContext) .analyze(testSql.get(1)) .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( logicalProject( @@ -165,7 +165,7 @@ public void testCaseJoinSameTable() { PlanChecker.from(connectContext) .analyze(testSql.get(5)) .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( logicalOlapScan(), diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java index c05ddc1bd065a2..b3387012f2ec0c 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/AnalyzeWhereSubqueryTest.java @@ -374,37 +374,39 @@ public void testSql10AfterAnalyze() { PlanChecker.from(connectContext) .analyze(sql10) .matchesFromRoot( - logicalProject( - logicalFilter( - logicalProject( - logicalApply( - any(), - logicalAggregate( - logicalSubQueryAlias( - logicalProject( - logicalFilter() - ).when(p -> p.getProjects().equals(ImmutableList.of( - new Alias(new ExprId(7), new SlotReference(new ExprId(5), "v1", BigIntType.INSTANCE, - true, - ImmutableList.of("default_cluster:test", "t7")), "aa") + logicalResultSink( + logicalProject( + logicalFilter( + logicalProject( + logicalApply( + any(), + logicalAggregate( + logicalSubQueryAlias( + logicalProject( + logicalFilter() + ).when(p -> p.getProjects().equals(ImmutableList.of( + new Alias(new ExprId(7), new SlotReference(new ExprId(5), "v1", BigIntType.INSTANCE, + true, + ImmutableList.of("default_cluster:test", "t7")), "aa") + ))) + ) + .when(a -> a.getAlias().equals("t2")) + .when(a -> a.getOutput().equals(ImmutableList.of( + new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, + true, ImmutableList.of("t2")) ))) - ) - .when(a -> a.getAlias().equals("t2")) - .when(a -> a.getOutput().equals(ImmutableList.of( - new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, - true, ImmutableList.of("t2")) + ).when(agg -> agg.getOutputExpressions().equals(ImmutableList.of( + new Alias(new ExprId(8), + (new Max(new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, + true, + ImmutableList.of("t2")))).withAlwaysNullable(true), "max(aa)") ))) - ).when(agg -> agg.getOutputExpressions().equals(ImmutableList.of( - new Alias(new ExprId(8), - (new Max(new SlotReference(new ExprId(7), "aa", BigIntType.INSTANCE, - true, - ImmutableList.of("t2")))).withAlwaysNullable(true), "max(aa)") - ))) - .when(agg -> agg.getGroupByExpressions().equals(ImmutableList.of())) + .when(agg -> agg.getGroupByExpressions().equals(ImmutableList.of())) + ) + .when(apply -> apply.getCorrelationSlot().equals(ImmutableList.of( + new SlotReference(new ExprId(1), "k2", BigIntType.INSTANCE, true, + ImmutableList.of("default_cluster:test", "t6"))))) ) - .when(apply -> apply.getCorrelationSlot().equals(ImmutableList.of( - new SlotReference(new ExprId(1), "k2", BigIntType.INSTANCE, true, - ImmutableList.of("default_cluster:test", "t6"))))) ) ) ) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java index 3a99e91dd84e29..af012ce3fb9f95 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/BindRelationTest.java @@ -115,7 +115,7 @@ public boolean hasDeleteSign() { .parse("select * from " + tableName + " as et join db1.t on et.id = t.a") .customAnalyzer(Optional.of(customTableResolver)) // analyze internal relation .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan().when(r -> r.getTable() == externalOlapTable), diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckExpressionLegalityTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckExpressionLegalityTest.java index 72fe56dcc8f86c..84e7cf4ff72c3b 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckExpressionLegalityTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/CheckExpressionLegalityTest.java @@ -56,11 +56,11 @@ public void testCountDistinctBitmap() { ConnectContext connectContext = MemoTestUtils.createConnectContext(); PlanChecker.from(connectContext) .analyze("select count(distinct id) from (select to_bitmap(1) id) tbl") - .matchesFromRoot(logicalAggregate().when(agg -> + .matches(logicalAggregate().when(agg -> agg.getOutputExpressions().get(0).child(0) instanceof Count )) .rewrite() - .matchesFromRoot(logicalAggregate().when(agg -> + .matches(logicalAggregate().when(agg -> agg.getOutputExpressions().get(0).child(0) instanceof BitmapUnionCount )); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java index 7cbafd8d5f4994..03cc549bc2c1fa 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FillUpMissingSlotsTest.java @@ -85,7 +85,7 @@ public void testHavingGroupBySlot() { ImmutableList.of("default_cluster:test_resolve_aggregate_functions", "t1") ); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalFilter( logicalAggregate( logicalOlapScan() @@ -99,7 +99,7 @@ public void testHavingGroupBySlot() { Alias value = new Alias(new ExprId(3), a1, "value"); PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -110,7 +110,7 @@ public void testHavingGroupBySlot() { sql = "SELECT a1 as value FROM t1 GROUP BY a1 HAVING value > 0"; PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) - .matchesFromRoot( + .matches( logicalFilter( logicalAggregate( logicalOlapScan() @@ -129,7 +129,7 @@ public void testHavingGroupBySlot() { Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)"); PlanChecker.from(connectContext).analyze(sql) .applyBottomUp(new ExpressionRewrite(FunctionBinder.INSTANCE)) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -152,7 +152,7 @@ public void testHavingAggregateFunction() { ); Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "sum(a2)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -164,7 +164,7 @@ public void testHavingAggregateFunction() { sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 HAVING SUM(a2) > 0"; sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -183,7 +183,7 @@ public void testHavingAggregateFunction() { ); Alias value = new Alias(new ExprId(3), new Sum(a2), "value"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -193,7 +193,7 @@ public void testHavingAggregateFunction() { sql = "SELECT a1, SUM(a2) as value FROM t1 GROUP BY a1 HAVING value > 0"; PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalFilter( logicalAggregate( logicalOlapScan() @@ -216,7 +216,7 @@ public void testHavingAggregateFunction() { ); Alias minPK = new Alias(new ExprId(4), new Min(pk), "min(pk)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -228,7 +228,7 @@ public void testHavingAggregateFunction() { sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 HAVING SUM(a1 + a2) > 0"; Alias sumA1A2 = new Alias(new ExprId(3), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -240,7 +240,7 @@ public void testHavingAggregateFunction() { Alias sumA1A23 = new Alias(new ExprId(4), new Sum(new Add(new Add(a1, a2), new TinyIntLiteral((byte) 3))), "sum(((a1 + a2) + 3))"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -252,7 +252,7 @@ public void testHavingAggregateFunction() { sql = "SELECT a1 FROM t1 GROUP BY a1 HAVING COUNT(*) > 0"; Alias countStar = new Alias(new ExprId(3), new Count(), "count(*)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -280,7 +280,7 @@ void testJoinWithHaving() { Alias sumA2 = new Alias(new ExprId(6), new Sum(a2), "sum(a2)"); Alias sumB1 = new Alias(new ExprId(7), new Sum(b1), "sum(b1)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -347,7 +347,7 @@ void testComplexQueryWithHaving() { Alias sumA1A2 = new Alias(new ExprId(11), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); Alias v1 = new Alias(new ExprId(12), new Count(a2), "v1"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalAggregate( @@ -388,7 +388,7 @@ public void testSortAggregateFunction() { ); Alias sumA2 = new Alias(new ExprId(3), new Sum(a2), "sum(a2)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( @@ -400,7 +400,7 @@ public void testSortAggregateFunction() { sql = "SELECT a1, SUM(a2) FROM t1 GROUP BY a1 ORDER BY SUM(a2)"; sumA2 = new Alias(new ExprId(3), new Sum(a2), "SUM(a2)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalSort( logicalAggregate( logicalOlapScan() @@ -418,7 +418,7 @@ public void testSortAggregateFunction() { ); Alias value = new Alias(new ExprId(3), new Sum(a2), "value"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalSort( logicalAggregate( logicalOlapScan() @@ -441,7 +441,7 @@ public void testSortAggregateFunction() { ); Alias minPK = new Alias(new ExprId(4), new Min(pk), "min(pk)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( @@ -453,7 +453,7 @@ public void testSortAggregateFunction() { sql = "SELECT a1, SUM(a1 + a2) FROM t1 GROUP BY a1 ORDER BY SUM(a1 + a2)"; Alias sumA1A2 = new Alias(new ExprId(3), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalSort( logicalAggregate( logicalOlapScan() @@ -464,7 +464,7 @@ public void testSortAggregateFunction() { Alias sumA1A23 = new Alias(new ExprId(4), new Sum(new Add(new Add(a1, a2), new TinyIntLiteral((byte) 3))), "sum(((a1 + a2) + 3))"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( @@ -476,7 +476,7 @@ public void testSortAggregateFunction() { sql = "SELECT a1 FROM t1 GROUP BY a1 ORDER BY COUNT(*)"; Alias countStar = new Alias(new ExprId(3), new Count(), "count(*)"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( @@ -511,7 +511,7 @@ void testComplexQueryWithOrderBy() { Alias sumA1A2 = new Alias(new ExprId(11), new Sum(new Add(a1, a2)), "SUM((a1 + a2))"); Alias v1 = new Alias(new ExprId(12), new Count(a2), "v1"); PlanChecker.from(connectContext).analyze(sql) - .matchesFromRoot( + .matches( logicalProject( logicalSort( logicalAggregate( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java index c7e32bed8194c2..9758a6c10b0d97 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/analysis/FunctionRegistryTest.java @@ -54,7 +54,7 @@ public void testDefaultFunctionNameIsClassName() { // and default class name should be year. PlanChecker.from(connectContext) .analyze("select year('2021-01-01')") - .matchesFromRoot( + .matches( logicalOneRowRelation().when(r -> { Year year = (Year) r.getProjects().get(0).child(0); Assertions.assertEquals("2021-01-01", @@ -71,7 +71,7 @@ public void testMultiName() { // 2. substr PlanChecker.from(connectContext) .analyze("select substring('abc', 1, 2), substr(substring('abcdefg', 4, 3), 1, 2)") - .matchesFromRoot( + .matches( logicalOneRowRelation().when(r -> { Substring firstSubstring = (Substring) r.getProjects().get(0).child(0); Assertions.assertEquals("abc", ((Literal) firstSubstring.getSource()).getValue()); @@ -94,7 +94,7 @@ public void testOverrideArity() { // 2. substring(string, position, length) PlanChecker.from(connectContext) .analyze("select substr('abc', 1), substring('def', 2, 3)") - .matchesFromRoot( + .matches( logicalOneRowRelation().when(r -> { Substring firstSubstring = (Substring) r.getProjects().get(0).child(0); Assertions.assertEquals("abc", ((Literal) firstSubstring.getSource()).getValue()); diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java index cad39fe234a54c..04e84ab8e89d86 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/ColumnPruningTest.java @@ -62,7 +62,7 @@ public void testPruneColumns1() { .analyze("select id,name,grade from student left join score on student.id = score.sid" + " where score.grade > 60") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalProject( @@ -94,7 +94,7 @@ public void testPruneColumns2() { + "from student left join score on student.id = score.sid " + "where score.grade > 60") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalProject( @@ -124,7 +124,7 @@ public void testPruneColumns3() { PlanChecker.from(connectContext) .analyze("select id,name from student where age > 18") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalProject().when(p -> getOutputQualifiedNames(p) @@ -146,7 +146,7 @@ public void testPruneColumns4() { + "on score.cid = course.cid " + "where score.grade > 60") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalFilter( logicalProject( @@ -184,7 +184,7 @@ public void pruneCountStarStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(*) FROM test.course") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -199,7 +199,7 @@ public void pruneCountConstantStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(1) FROM test.course") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -214,7 +214,7 @@ public void pruneCountConstantAndSumConstantStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(1), SUM(2) FROM test.course") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -229,7 +229,7 @@ public void pruneCountStarAndSumConstantStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(*), SUM(2) FROM test.course") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -244,7 +244,7 @@ public void pruneCountStarAndSumColumnStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(*), SUM(grade) FROM test.score") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -259,7 +259,7 @@ public void pruneCountStarAndSumColumnAndSumConstantStmt() { PlanChecker.from(connectContext) .analyze("SELECT COUNT(*), SUM(grade) + SUM(2) FROM test.score") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalAggregate( logicalProject( logicalOlapScan() @@ -274,7 +274,7 @@ public void pruneColumnForOneSideOnCrossJoin() { PlanChecker.from(connectContext) .analyze("select id,name from student cross join score") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject(logicalRelation()) @@ -296,7 +296,7 @@ public void pruneAggregateOutput() { PlanChecker.from(connectContext) .analyze("select id from (select id, sum(age) from student group by id)a") .customRewrite(new ColumnPruning()) - .matchesFromRoot( + .matches( logicalProject( logicalSubQueryAlias( logicalAggregate( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java index 1203f542f4f121..04613f7e75e7f0 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/InferPredicatesTest.java @@ -81,7 +81,7 @@ public void inferPredicatesTest01() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -102,7 +102,7 @@ public void inferPredicatesTest02() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan(), @@ -119,7 +119,7 @@ public void inferPredicatesTest03() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -138,7 +138,7 @@ public void inferPredicatesTest04() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -157,7 +157,7 @@ public void inferPredicatesTest05() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalJoin( @@ -183,7 +183,7 @@ public void inferPredicatesTest06() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalJoin( @@ -209,7 +209,7 @@ public void inferPredicatesTest07() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -230,7 +230,7 @@ public void inferPredicatesTest08() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan(), @@ -250,7 +250,7 @@ public void inferPredicatesTest09() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -271,7 +271,7 @@ public void inferPredicatesTest10() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -294,7 +294,7 @@ public void inferPredicatesTest11() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -315,7 +315,7 @@ public void inferPredicatesTest12() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -339,7 +339,7 @@ public void inferPredicatesTest13() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -362,7 +362,7 @@ public void inferPredicatesTest14() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -385,7 +385,7 @@ public void inferPredicatesTest15() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -408,7 +408,7 @@ public void inferPredicatesTest16() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan(), @@ -429,7 +429,7 @@ public void inferPredicatesTest17() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalOlapScan(), @@ -450,7 +450,7 @@ public void inferPredicatesTest18() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -495,7 +495,7 @@ public void inferPredicatesTest19() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( @@ -534,7 +534,7 @@ public void inferPredicatesTest20() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( innerLogicalJoin( @@ -560,7 +560,7 @@ public void inferPredicatesTest21() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalJoin( @@ -589,7 +589,7 @@ public void inferPredicatesTest22() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalFilter( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java index 7d9114cef3279b..29cc509d954988 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/PushdownExpressionsInHashConditionTest.java @@ -105,7 +105,7 @@ public void testSubQueryCase() { "SELECT * FROM (SELECT * FROM T1) X JOIN (SELECT * FROM T2) Y ON X.ID + 1 = Y.ID + 2 AND X.ID + 1 > 2") .applyTopDown(new FindHashConditionForJoin()) .applyTopDown(new PushdownExpressionsInHashCondition()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -134,7 +134,7 @@ public void testAggNodeCase() { "SELECT * FROM T1 JOIN (SELECT ID, SUM(SCORE) SCORE FROM T2 GROUP BY ID) T ON T1.ID + 1 = T.ID AND T.SCORE = T1.SCORE + 10") .applyTopDown(new FindHashConditionForJoin()) .applyTopDown(new PushdownExpressionsInHashCondition()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( @@ -159,7 +159,7 @@ public void testSortNodeCase() { "SELECT * FROM T1 JOIN (SELECT ID, SUM(SCORE) SCORE FROM T2 GROUP BY ID ORDER BY ID) T ON T1.ID + 1 = T.ID AND T.SCORE = T1.SCORE + 10") .applyTopDown(new FindHashConditionForJoin()) .applyTopDown(new PushdownExpressionsInHashCondition()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java index 2342d8f14a7337..a3bd46eb4f2f4e 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/mv/SelectRollupIndexTest.java @@ -261,6 +261,7 @@ public void testKeyColumnInAggFunction() { })); } + @Disabled("reopen it if we fix rollup select bugs") @Test public void testMaxCanUseKeyColumn() { PlanChecker.from(connectContext) @@ -275,6 +276,7 @@ public void testMaxCanUseKeyColumn() { })); } + @Disabled("reopen it if we fix rollup select bugs") @Test public void testMinCanUseKeyColumn() { PlanChecker.from(connectContext) diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java index adc5a993fdaf66..43c8c4b9dc42e7 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/InferTest.java @@ -29,7 +29,7 @@ void testInferNotNullAndInferPredicates() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( logicalFilter().when(f -> f.getPredicate().toString().equals("(id#0 = 4)")), @@ -47,7 +47,7 @@ void testInferNotNullFromFilterAndEliminateOuter2() { .analyze(sql) .rewrite() .printlnTree() - .matchesFromRoot( + .matches( logicalProject( innerLogicalJoin( logicalFilter().when( @@ -65,7 +65,7 @@ void testInferNotNullFromFilterAndEliminateOuter3() { PlanChecker.from(connectContext) .analyze(sql) .rewrite() - .matchesFromRoot( + .matches( logicalProject( logicalFilter( leftOuterLogicalJoin( diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/JoinTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/JoinTest.java index aa5514d304c5b7..72f8ec0879758d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/JoinTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/sqltest/JoinTest.java @@ -17,6 +17,7 @@ package org.apache.doris.nereids.sqltest; +import org.apache.doris.nereids.properties.DistributionSpecGather; import org.apache.doris.nereids.properties.DistributionSpecHash; import org.apache.doris.nereids.properties.DistributionSpecHash.ShuffleType; import org.apache.doris.nereids.rules.rewrite.ReorderJoin; @@ -49,7 +50,12 @@ void testColocatedJoin() { .getBestPlanTree(); // generate colocate join plan without physicalDistribute System.out.println(plan.treeString()); - Assertions.assertFalse(plan.anyMatch(PhysicalDistribute.class::isInstance)); + Assertions.assertFalse(plan.anyMatch(p -> { + if (p instanceof PhysicalDistribute) { + return !(((PhysicalDistribute) p).getDistributionSpec() instanceof DistributionSpecGather); + } + return false; + })); sql = "select * from T1 join T0 on T1.score = T0.score and T1.id = T0.id;"; plan = PlanChecker.from(connectContext) .analyze(sql) @@ -57,7 +63,12 @@ void testColocatedJoin() { .optimize() .getBestPlanTree(); // generate colocate join plan without physicalDistribute - Assertions.assertFalse(plan.anyMatch(PhysicalDistribute.class::isInstance)); + Assertions.assertFalse(plan.anyMatch(p -> { + if (p instanceof PhysicalDistribute) { + return !(((PhysicalDistribute) p).getDistributionSpec() instanceof DistributionSpecGather); + } + return false; + })); } @Test @@ -91,7 +102,9 @@ void testBucketJoinWithAgg() { .optimize() .getBestPlanTree(); Assertions.assertEquals( - ((DistributionSpecHash) plan.getPhysicalProperties().getDistributionSpec()).getShuffleType(), - ShuffleType.NATURAL); + ShuffleType.NATURAL, + ((DistributionSpecHash) ((PhysicalPlan) (plan.child(0).child(0))) + .getPhysicalProperties().getDistributionSpec()).getShuffleType() + ); } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ViewTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ViewTest.java index 8bb4dc81afaafa..4fe217b0e6bff8 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ViewTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/expressions/ViewTest.java @@ -115,7 +115,7 @@ public void testSimpleViewMergeProjects() { .analyze("SELECT * FROM V1") .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) .applyTopDown(new MergeProjects()) - .matchesFromRoot( + .matches( logicalProject( logicalOlapScan() ) @@ -142,7 +142,7 @@ public void testNestedView() { ) .applyTopDown(new LogicalSubQueryAliasToLogicalProject()) .applyTopDown(new MergeProjects()) - .matchesFromRoot( + .matches( logicalProject( logicalJoin( logicalProject( diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out index 0a15a57002d99f..55ff9ac886d73d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query1.out @@ -13,29 +13,30 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalProject ------------------filter((date_dim.d_year = 2000)) --------------------PhysicalOlapScan[date_dim] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) -------------PhysicalDistribute ---------------PhysicalProject -----------------PhysicalOlapScan[customer] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) -------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) ---------------------PhysicalDistribute -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((cast(s_state as VARCHAR(*)) = 'SD')) ---------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +--------------PhysicalDistribute +----------------PhysicalProject +------------------PhysicalOlapScan[customer] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_store_sk = ctr2.ctr_store_sk)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +--------------------hashJoin[INNER_JOIN](store.s_store_sk = ctr1.ctr_store_sk) +----------------------PhysicalDistribute +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalProject +--------------------------filter((cast(s_state as VARCHAR(*)) = 'SD')) +----------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out index 1f82006aec7845..2ec618b7da99f8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query10.out @@ -1,56 +1,57 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_10 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(($c$1 OR $c$2)) -------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) ---------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) -----------------------------PhysicalOlapScan[customer_demographics] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(($c$1 OR $c$2)) +--------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) +----------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +------------------------------PhysicalOlapScan[customer_demographics] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy <= 4)(date_dim.d_moy >= 1)(date_dim.d_year = 2001)) +------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalOlapScan[customer] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_moy <= 4)(date_dim.d_moy >= 1)(date_dim.d_year = 2001)) -----------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter(ca_county IN ('Storey County', 'Marquette County', 'Warren County', 'Cochran County', 'Kandiyohi County')) ---------------------------------------------PhysicalOlapScan[customer_address] +--------------------------------------------filter(ca_county IN ('Storey County', 'Marquette County', 'Warren County', 'Cochran County', 'Kandiyohi County')) +----------------------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy <= 4)(date_dim.d_moy >= 1)(date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy <= 4)(date_dim.d_moy >= 1)(date_dim.d_year = 2001)) +--------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2001)(date_dim.d_moy <= 4)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2001)(date_dim.d_moy <= 4)) ---------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out index b97d80540ed59b..e773a1bf5fc02b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query11.out @@ -40,27 +40,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter('w' IN ('s', 'w')) ------------------------PhysicalOlapScan[customer] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END) -------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) ---------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END > CASE WHEN (year_total > 0.00) THEN (cast(year_total as DECIMALV3(38, 8)) / year_total) ELSE 0.000000 END) +--------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) +----------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_s_firstyear.dyear = 2001)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.00)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((t_w_firstyear.year_total > 0.00)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.dyear = 2001)) +--------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002)) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2002)) +------------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w')) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((t_w_secyear.dyear = 2002)(t_w_secyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out index 5750cb0006f408..a48c3cceb8f30a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query12.out @@ -1,27 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_12 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_date <= 1998-05-06)(date_dim.d_date >= 1998-04-06)) -----------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(i_category IN ('Books', 'Sports', 'Men')) -------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_date <= 1998-05-06)(date_dim.d_date >= 1998-04-06)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter(i_category IN ('Books', 'Sports', 'Men')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out index 7e06af0ebffb69..e1ae820d98e6c1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query13.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_13 -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------PhysicalProject -------------PhysicalOlapScan[store] -----------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------PhysicalProject -------------------filter((date_dim.d_year = 2001)) ---------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute +--------------PhysicalOlapScan[store] +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)((((((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)) OR ((((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) AND (household_demographics.hd_dep_count = 1))) OR ((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))) AND (household_demographics.hd_dep_count = 1))) -----------------------PhysicalProject -------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'College'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')))) ---------------------------PhysicalOlapScan[customer_demographics] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------PhysicalProject -----------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_dep_count = 1))) -------------------------------PhysicalOlapScan[household_demographics] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('SD', 'KS', 'MI') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('MO', 'ND', 'CO') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))) -------------------------------PhysicalProject ---------------------------------filter(((((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) -----------------------------------PhysicalOlapScan[store_sales] -------------------------------PhysicalDistribute +--------------------filter((date_dim.d_year = 2001)) +----------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)((((((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) AND (household_demographics.hd_dep_count = 3)) OR ((((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) AND (household_demographics.hd_dep_count = 1))) OR ((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00))) AND (household_demographics.hd_dep_count = 1))) +------------------------PhysicalProject +--------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'College'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')))) +----------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------PhysicalProject +------------------------------filter(((household_demographics.hd_dep_count = 3) OR (household_demographics.hd_dep_count = 1))) +--------------------------------PhysicalOlapScan[household_demographics] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('SD', 'KS', 'MI') AND ((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00))) OR (ca_state IN ('MO', 'ND', 'CO') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00)))) OR (ca_state IN ('NH', 'OH', 'TX') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))) --------------------------------PhysicalProject -----------------------------------filter((customer_address.ca_country = 'United States')((ca_state IN ('SD', 'KS', 'MI') OR ca_state IN ('MO', 'ND', 'CO')) OR ca_state IN ('NH', 'OH', 'TX'))) -------------------------------------PhysicalOlapScan[customer_address] +----------------------------------filter(((((store_sales.ss_net_profit >= 100.00) AND (store_sales.ss_net_profit <= 200.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 300.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 250.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) +------------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((customer_address.ca_country = 'United States')((ca_state IN ('SD', 'KS', 'MI') OR ca_state IN ('MO', 'ND', 'CO')) OR ca_state IN ('NH', 'OH', 'TX'))) +--------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query14.out index 04a155f26ccf04..5a7bdf71a5abaa 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 @@ -67,91 +67,92 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------PhysicalProject ------------------------filter((date_dim.d_year >= 2000)(date_dim.d_year <= 2002)) --------------------------PhysicalOlapScan[date_dim] -----PhysicalTopN -------PhysicalDistribute ---------PhysicalTopN -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalRepeat ---------------------PhysicalUnion -----------------------PhysicalProject -------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) ---------------------------PhysicalProject -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----PhysicalResultSink +------PhysicalTopN +--------PhysicalDistribute +----------PhysicalTopN +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalRepeat +----------------------PhysicalUnion +------------------------PhysicalProject +--------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[RIGHT_SEMI_JOIN](store_sales.ss_item_sk = cross_items.ss_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------------PhysicalDistribute +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -----------------------PhysicalProject -------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) ---------------------------PhysicalProject -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +----------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) ---------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -----------------------PhysicalProject -------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) ---------------------------PhysicalProject -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalAssertNumRows +--------------------------------PhysicalDistribute +----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------PhysicalProject +--------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = cross_items.ss_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------------PhysicalDistribute +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) ---------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +----------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalAssertNumRows +--------------------------------PhysicalDistribute +----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------PhysicalProject +--------------------------NestedLoopJoin[INNER_JOIN](cast(sales as DOUBLE) > cast(average_sales as DOUBLE)) +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = cross_items.ss_item_sk) ----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalAssertNumRows -------------------------------PhysicalDistribute ---------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +------------------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[web_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 11)) +----------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalAssertNumRows +--------------------------------PhysicalDistribute +----------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out index 9d365f0d826ea6..abb2fe6156f5ae 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query15.out @@ -1,27 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_15 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR ca_state IN ('CA', 'WA', 'GA')) OR (catalog_sales.cs_sales_price > 500.00)) -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[catalog_sales] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((date_dim.d_qoy = 1)(date_dim.d_year = 2001)) ---------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk)((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR ca_state IN ('CA', 'WA', 'GA')) OR (catalog_sales.cs_sales_price > 500.00)) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalOlapScan[catalog_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] +--------------------------filter((date_dim.d_qoy = 1)(date_dim.d_year = 2001)) +----------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out index 550b3e1360345f..967e3b60636fcb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query16.out @@ -1,38 +1,39 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_16 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----PhysicalProject -------hashAgg[GLOBAL] ---------hashAgg[LOCAL] +----PhysicalTopN +------PhysicalProject +--------hashAgg[GLOBAL] ----------PhysicalDistribute -------------PhysicalProject ---------------hashJoin[INNER_JOIN](cs1.cs_call_center_sk = call_center.cc_call_center_sk) -----------------PhysicalProject -------------------filter(cc_county IN ('Ziebach County', 'Luce County', 'Richland County', 'Daviess County', 'Barrow County')) ---------------------PhysicalOlapScan[call_center] -----------------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](cs1.cs_call_center_sk = call_center.cc_call_center_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](cs1.cs_ship_date_sk = date_dim.d_date_sk) -----------------------PhysicalProject -------------------------filter((cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('2002-4-01' as DATE), INTERVAL 60 DAY) as DATETIMEV2(0)))(date_dim.d_date >= 2002-04-01)) ---------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[RIGHT_SEMI_JOIN](cs1.cs_order_number = cs2.cs_order_number)( not (cs_warehouse_sk = cs_warehouse_sk)) +--------------------filter(cc_county IN ('Ziebach County', 'Luce County', 'Richland County', 'Daviess County', 'Barrow County')) +----------------------PhysicalOlapScan[call_center] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[RIGHT_SEMI_JOIN](cs1.cs_order_number = cs2.cs_order_number)( not (cs_warehouse_sk = cs_warehouse_sk)) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](cs1.cs_ship_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------filter((cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('2002-4-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))(date_dim.d_date >= 2002-04-01)) +--------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] -----------------------------hashJoin[RIGHT_ANTI_JOIN](cs1.cs_order_number = cr1.cr_order_number) -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_returns] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](cs1.cs_ship_addr_sk = customer_address.ca_address_sk) +------------------------------hashJoin[RIGHT_ANTI_JOIN](cs1.cs_order_number = cr1.cr_order_number) +--------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute +------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](cs1.cs_ship_addr_sk = customer_address.ca_address_sk) ------------------------------------PhysicalProject ---------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'WV')) -----------------------------------------PhysicalOlapScan[customer_address] +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'WV')) +------------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out index 7a6ac4e43780f7..ab873f067427af 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query17.out @@ -1,47 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_17 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[store] -------------------PhysicalDistribute ---------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +--------------------------PhysicalOlapScan[catalog_sales] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ----------------------------PhysicalProject -------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) ---------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) +------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +--------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) +------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_returns] +------------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_returns] ---------------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +----------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------------PhysicalDistribute ----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((cast(d_quarter_name as VARCHAR(*)) = '2001Q1')) -----------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------------PhysicalDistribute -------------------------------------------PhysicalProject ---------------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) -----------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------filter((cast(d_quarter_name as VARCHAR(*)) = '2001Q1')) +--------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(d_quarter_name IN ('2001Q1', '2001Q2', '2001Q3')) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out index e2953ab328518f..44c121660a3fe1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query18.out @@ -1,48 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_18 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer.c_current_cdemo_sk = cd2.cd_demo_sk) ---------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_current_cdemo_sk = cd2.cd_demo_sk) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_demographics] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk) +------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +----------------------------------PhysicalDistribute ------------------------------------PhysicalProject --------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter(c_birth_month IN (1, 2, 4, 7, 8, 10)) ---------------------------------------------------PhysicalOlapScan[customer] +------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------------------------PhysicalProject +----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = cd1.cd_demo_sk) +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------------------PhysicalDistribute +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((cast(cd_gender as VARCHAR(*)) = 'F')(cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')) +------------------------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------------------PhysicalDistribute ----------------------------------------------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')) -------------------------------------------PhysicalOlapScan[customer_demographics] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------filter((date_dim.d_year = 1998)) +--------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter(c_birth_month IN (1, 2, 4, 7, 8, 10)) +----------------------------------------------PhysicalOlapScan[customer] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter(ca_state IN ('WA', 'GA', 'NC', 'ME', 'WY', 'OK', 'IN')) +----------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out index 5a6299952a9ddf..b3b38025535b7a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query19.out @@ -1,38 +1,39 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_19 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)( not (substring(ca_zip, 1, 5) = substring(s_zip, 1, 5))) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk)( not (substring(ca_zip, 1, 5) = substring(s_zip, 1, 5))) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((item.i_manager_id = 2)) +--------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((item.i_manager_id = 2)) -------------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] +----------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out index 4d4ce2d58c8081..d05df91baef121 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query2.out @@ -17,29 +17,30 @@ PhysicalCteAnchor ( cteId=CTEId#1 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------PhysicalOlapScan[date_dim] ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------hashJoin[INNER_JOIN](expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53)) -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq2) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_year = 1999)) -------------------------PhysicalOlapScan[date_dim] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq1) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_year = 1998)) -------------------------PhysicalOlapScan[date_dim] +--PhysicalResultSink +----PhysicalQuickSort +------PhysicalDistribute +--------PhysicalQuickSort +----------PhysicalProject +------------hashJoin[INNER_JOIN](expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 53)) +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq2) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((date_dim.d_year = 1999)) +--------------------------PhysicalOlapScan[date_dim] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = d_week_seq1) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((date_dim.d_year = 1998)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out index ef38ef4273ab70..8db8714815e2b6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query20.out @@ -1,26 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_20 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date >= 2002-01-26)(date_dim.d_date <= 2002-02-25)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_date >= 2002-01-26)(date_dim.d_date <= 2002-02-25)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(i_category IN ('Shoes', 'Books', 'Women')) -------------------------------PhysicalOlapScan[item] +------------------------------filter(i_category IN ('Shoes', 'Books', 'Women')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out index 32df59522602e3..f950635ad66931 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query21.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_21 -- -PhysicalTopN ---PhysicalDistribute -----filter((CASE WHEN (inv_before > 0) THEN (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)) ELSE NULL END >= cast((2.000000 / 3.0) as DOUBLE))(CASE WHEN (inv_before > 0) THEN (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)) ELSE NULL END <= 1.5)) -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](inventory.inv_warehouse_sk = warehouse.w_warehouse_sk) -----------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) -------------------hashJoin[INNER_JOIN](item.i_item_sk = inventory.inv_item_sk) ---------------------PhysicalOlapScan[inventory] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------filter((CASE WHEN (inv_before > 0) THEN (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)) ELSE NULL END >= cast((2.000000 / 3.0) as DOUBLE))(CASE WHEN (inv_before > 0) THEN (cast(inv_after as DOUBLE) / cast(inv_before as DOUBLE)) ELSE NULL END <= 1.5)) +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](inventory.inv_warehouse_sk = warehouse.w_warehouse_sk) +------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) +--------------------hashJoin[INNER_JOIN](item.i_item_sk = inventory.inv_item_sk) +----------------------PhysicalOlapScan[inventory] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_current_price <= 1.49)(item.i_current_price >= 0.99)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_current_price <= 1.49)(item.i_current_price >= 0.99)) ---------------------------PhysicalOlapScan[item] +------------------------filter((date_dim.d_date >= 2002-01-28)(date_dim.d_date <= 2002-03-29)) +--------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((date_dim.d_date >= 2002-01-28)(date_dim.d_date <= 2002-03-29)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out index a5d6f602e671f8..ec560939d191a6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query22.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_22 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) ---------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[inventory] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = date_dim.d_date_sk) +----------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[inventory] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((date_dim.d_month_seq <= 1199)(date_dim.d_month_seq >= 1188)) ---------------------------PhysicalOlapScan[date_dim] +--------------------------filter((date_dim.d_month_seq <= 1199)(date_dim.d_month_seq >= 1188)) +----------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out index 1deef474c9d053..9f1ec957297b6d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query23.out @@ -55,48 +55,49 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject --------------------------------------PhysicalOlapScan[customer] -----PhysicalLimit +----PhysicalResultSink ------PhysicalLimit ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalUnion -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = frequent_ss_items.item_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] -------------------------------PhysicalDistribute +--------PhysicalLimit +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalUnion +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN](catalog_sales.cs_item_sk = frequent_ss_items.item_sk) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN](catalog_sales.cs_bill_customer_sk = best_ss_customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) -----------------PhysicalProject -------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = frequent_ss_items.item_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[LEFT_SEMI_JOIN](web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) +--------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] -------------------------------PhysicalDistribute +--------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +------------------PhysicalProject +--------------------hashJoin[RIGHT_SEMI_JOIN](web_sales.ws_item_sk = frequent_ss_items.item_sk) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[LEFT_SEMI_JOIN](web_sales.ws_bill_customer_sk = best_ss_customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------PhysicalProject -----------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) -------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) +----------------------------------PhysicalOlapScan[web_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 5)) +--------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#2 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out index eebe6647279d9d..a9e8df2aaef283 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query24.out @@ -31,25 +31,26 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------------PhysicalOlapScan[item] ----------------PhysicalProject ------------------PhysicalOlapScan[store_returns] ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------NestedLoopJoin[INNER_JOIN](cast(paid as DOUBLE) > cast((0.05 * avg(netpaid)) as DOUBLE)) -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((cast(i_color as VARCHAR(*)) = 'beige')) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalAssertNumRows -----------------PhysicalProject -------------------hashAgg[GLOBAL] +--PhysicalResultSink +----PhysicalQuickSort +------PhysicalDistribute +--------PhysicalQuickSort +----------PhysicalProject +------------NestedLoopJoin[INNER_JOIN](cast(paid as DOUBLE) > cast((0.05 * avg(netpaid)) as DOUBLE)) +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] --------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalProject +------------------------filter((cast(i_color as VARCHAR(*)) = 'beige')) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------PhysicalAssertNumRows +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out index 83877b55505e32..35343e9a8c9146 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query25.out @@ -1,46 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_25 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) +------------------------PhysicalOlapScan[catalog_sales] +----------------------PhysicalDistribute +------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) --------------------------PhysicalProject -----------------------------filter((d3.d_year = 2000)(d3.d_moy <= 10)(d3.d_moy >= 4)) -------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) +----------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) ---------------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) +------------------------------------------PhysicalOlapScan[store_returns] +----------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute --------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_returns] ---------------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------------------------PhysicalProject -------------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------------PhysicalDistribute -------------------------------------------------PhysicalProject ---------------------------------------------------filter((d1.d_year = 2000)(d1.d_moy = 4)) -----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------filter((d1.d_year = 2000)(d1.d_moy = 4)) +------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter((d2.d_moy <= 10)(d2.d_moy >= 4)(d2.d_year = 2000)) ------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((d3.d_year = 2000)(d3.d_moy <= 10)(d3.d_moy >= 4)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out index 9a85f030a23990..cdffd94d108e07 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query26.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_26 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((cast(cd_education_status as VARCHAR(*)) = 'Unknown')(cast(cd_gender as VARCHAR(*)) = 'M')(cast(cd_marital_status as VARCHAR(*)) = 'S')) +----------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((cast(cd_education_status as VARCHAR(*)) = 'Unknown')(cast(cd_gender as VARCHAR(*)) = 'M')(cast(cd_marital_status as VARCHAR(*)) = 'S')) ---------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) ---------------------------PhysicalOlapScan[promotion] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) +----------------------------PhysicalOlapScan[promotion] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out index e773eae3049644..81fad1aff1d21e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query27.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_27 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------PhysicalRepeat -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------PhysicalRepeat +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((cast(cd_gender as VARCHAR(*)) = 'F')(cast(cd_education_status as VARCHAR(*)) = 'Secondary')(cast(cd_marital_status as VARCHAR(*)) = 'D')) +----------------------------------------PhysicalOlapScan[customer_demographics] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((cast(cd_gender as VARCHAR(*)) = 'F')(cast(cd_education_status as VARCHAR(*)) = 'Secondary')(cast(cd_marital_status as VARCHAR(*)) = 'D')) ---------------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((date_dim.d_year = 1999)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------------------filter((date_dim.d_year = 1999)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter(s_state IN ('MO', 'AL', 'MI', 'TN', 'LA', 'SC')) -----------------------------PhysicalOlapScan[store] +----------------------------filter(s_state IN ('MO', 'AL', 'MI', 'TN', 'LA', 'SC')) +------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out index 7a07c8ac875955..cf9431b2e02bf7 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query28.out @@ -1,90 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_28 -- -PhysicalLimit +PhysicalResultSink --PhysicalLimit -----PhysicalProject -------NestedLoopJoin[CROSS_JOIN] ---------PhysicalLimit +----PhysicalLimit +------PhysicalProject +--------NestedLoopJoin[CROSS_JOIN] ----------PhysicalLimit ------------NestedLoopJoin[CROSS_JOIN] --------------PhysicalLimit -----------------PhysicalLimit -------------------NestedLoopJoin[CROSS_JOIN] ---------------------PhysicalLimit +----------------NestedLoopJoin[CROSS_JOIN] +------------------PhysicalLimit +--------------------NestedLoopJoin[CROSS_JOIN] ----------------------PhysicalLimit ------------------------NestedLoopJoin[CROSS_JOIN] --------------------------PhysicalLimit -----------------------------PhysicalLimit -------------------------------NestedLoopJoin[CROSS_JOIN] ---------------------------------PhysicalLimit -----------------------------------PhysicalLimit -------------------------------------hashAgg[DISTINCT_GLOBAL] ---------------------------------------PhysicalDistribute -----------------------------------------hashAgg[DISTINCT_LOCAL] -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter((store_sales.ss_quantity <= 5)((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00)))(store_sales.ss_quantity >= 0)) -----------------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------PhysicalDistribute -----------------------------------PhysicalLimit -------------------------------------PhysicalLimit ---------------------------------------hashAgg[DISTINCT_GLOBAL] -----------------------------------------PhysicalDistribute -------------------------------------------hashAgg[DISTINCT_LOCAL] ---------------------------------------------hashAgg[GLOBAL] -----------------------------------------------PhysicalDistribute -------------------------------------------------hashAgg[LOCAL] ---------------------------------------------------PhysicalProject -----------------------------------------------------filter((store_sales.ss_quantity <= 10)((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00)))(store_sales.ss_quantity >= 6)) -------------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((store_sales.ss_quantity <= 5)((((store_sales.ss_list_price >= 131.00) AND (store_sales.ss_list_price <= 141.00)) OR ((store_sales.ss_coupon_amt >= 16798.00) AND (store_sales.ss_coupon_amt <= 17798.00))) OR ((store_sales.ss_wholesale_cost >= 25.00) AND (store_sales.ss_wholesale_cost <= 45.00)))(store_sales.ss_quantity >= 0)) +--------------------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalLimit -------------------------------PhysicalLimit ---------------------------------hashAgg[DISTINCT_GLOBAL] -----------------------------------PhysicalDistribute -------------------------------------hashAgg[DISTINCT_LOCAL] ---------------------------------------hashAgg[GLOBAL] -----------------------------------------PhysicalDistribute -------------------------------------------hashAgg[LOCAL] ---------------------------------------------PhysicalProject -----------------------------------------------filter(((((store_sales.ss_list_price >= 1.5E+2) AND (store_sales.ss_list_price <= 1.6E+2)) OR ((store_sales.ss_coupon_amt >= 6.6E+3) AND (store_sales.ss_coupon_amt <= 7.6E+3))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00)))(store_sales.ss_quantity >= 11)(store_sales.ss_quantity <= 15)) -------------------------------------------------PhysicalOlapScan[store_sales] ---------------------PhysicalDistribute -----------------------PhysicalLimit -------------------------PhysicalLimit ---------------------------hashAgg[DISTINCT_GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[DISTINCT_LOCAL] ---------------------------------hashAgg[GLOBAL] -----------------------------------PhysicalDistribute -------------------------------------hashAgg[LOCAL] ---------------------------------------PhysicalProject -----------------------------------------filter((store_sales.ss_quantity <= 20)((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00)))(store_sales.ss_quantity >= 16)) -------------------------------------------PhysicalOlapScan[store_sales] ---------------PhysicalDistribute -----------------PhysicalLimit -------------------PhysicalLimit ---------------------hashAgg[DISTINCT_GLOBAL] +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------filter((store_sales.ss_quantity <= 10)((((store_sales.ss_list_price >= 145.00) AND (store_sales.ss_list_price <= 155.00)) OR ((store_sales.ss_coupon_amt >= 14792.00) AND (store_sales.ss_coupon_amt <= 15792.00))) OR ((store_sales.ss_wholesale_cost >= 46.00) AND (store_sales.ss_wholesale_cost <= 66.00)))(store_sales.ss_quantity >= 6)) +----------------------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute -------------------------hashAgg[DISTINCT_LOCAL] +------------------------PhysicalLimit --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00)))(store_sales.ss_quantity <= 25)(store_sales.ss_quantity >= 21)) +----------------------------------filter(((((store_sales.ss_list_price >= 1.5E+2) AND (store_sales.ss_list_price <= 1.6E+2)) OR ((store_sales.ss_coupon_amt >= 6.6E+3) AND (store_sales.ss_coupon_amt <= 7.6E+3))) OR ((store_sales.ss_wholesale_cost >= 9.00) AND (store_sales.ss_wholesale_cost <= 29.00)))(store_sales.ss_quantity >= 11)(store_sales.ss_quantity <= 15)) ------------------------------------PhysicalOlapScan[store_sales] ---------PhysicalDistribute -----------PhysicalLimit +------------------PhysicalDistribute +--------------------PhysicalLimit +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((store_sales.ss_quantity <= 20)((((store_sales.ss_list_price >= 91.00) AND (store_sales.ss_list_price <= 101.00)) OR ((store_sales.ss_coupon_amt >= 13493.00) AND (store_sales.ss_coupon_amt <= 14493.00))) OR ((store_sales.ss_wholesale_cost >= 36.00) AND (store_sales.ss_wholesale_cost <= 56.00)))(store_sales.ss_quantity >= 16)) +--------------------------------PhysicalOlapScan[store_sales] +--------------PhysicalDistribute +----------------PhysicalLimit +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------filter(((((store_sales.ss_list_price >= 0.00) AND (store_sales.ss_list_price <= 10.00)) OR ((store_sales.ss_coupon_amt >= 7629.00) AND (store_sales.ss_coupon_amt <= 8629.00))) OR ((store_sales.ss_wholesale_cost >= 6.00) AND (store_sales.ss_wholesale_cost <= 26.00)))(store_sales.ss_quantity <= 25)(store_sales.ss_quantity >= 21)) +----------------------------PhysicalOlapScan[store_sales] +----------PhysicalDistribute ------------PhysicalLimit ---------------hashAgg[DISTINCT_GLOBAL] +--------------hashAgg[GLOBAL] ----------------PhysicalDistribute -------------------hashAgg[DISTINCT_LOCAL] ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------filter((store_sales.ss_quantity >= 26)((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00)))(store_sales.ss_quantity <= 30)) -------------------------------PhysicalOlapScan[store_sales] +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------filter((store_sales.ss_quantity >= 26)((((store_sales.ss_list_price >= 89.00) AND (store_sales.ss_list_price <= 99.00)) OR ((store_sales.ss_coupon_amt >= 15257.00) AND (store_sales.ss_coupon_amt <= 16257.00))) OR ((store_sales.ss_wholesale_cost >= 31.00) AND (store_sales.ss_wholesale_cost <= 51.00)))(store_sales.ss_quantity <= 30)) +------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out index 95be784e2a73c2..5f93ae5559fe90 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query29.out @@ -1,46 +1,47 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_29 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +------------------PhysicalProject +--------------------PhysicalOlapScan[store] +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) ---------------------------PhysicalProject -----------------------------filter(d_year IN (1999, 2000, 2001)) -------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute +------------------------PhysicalOlapScan[item] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d3.d_date_sk) ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------PhysicalDistribute +------------------------------filter(d_year IN (1999, 2000, 2001)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = catalog_sales.cs_item_sk)(store_returns.sr_customer_sk = catalog_sales.cs_bill_customer_sk) ----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[store_returns] -------------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) --------------------------------------------PhysicalProject ----------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) ----------------------------------------------PhysicalProject -------------------------------------------------filter((d1.d_year = 1999)(d1.d_moy = 4)) ---------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((d2.d_moy <= 7)(d2.d_moy >= 4)(d2.d_year = 1999)) ---------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------------------PhysicalOlapScan[store_returns] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((d2.d_moy <= 7)(d2.d_moy >= 4)(d2.d_year = 1999)) +----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((d1.d_year = 1999)(d1.d_moy = 4)) +----------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out index c11369d38f4287..6516eca4168b97 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query3.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_3 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_manufact_id = 816)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_manufact_id = 816)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((dt.d_moy = 11)) -------------------------PhysicalOlapScan[date_dim] +------------------------filter((dt.d_moy = 11)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out index 7741e0c45cd144..48239bce9e9f72 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query30.out @@ -19,28 +19,29 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) -------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) ---------------PhysicalDistribute -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +--------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +----------------PhysicalDistribute +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) --------------------PhysicalProject -----------------------filter((cast(ca_state as VARCHAR(*)) = 'IN')) -------------------------PhysicalOlapScan[customer_address] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((cast(ca_state as VARCHAR(*)) = 'IN')) +--------------------------PhysicalOlapScan[customer_address] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out index dd60ee2d41baf1..ed3787c35062ab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query31.out @@ -39,38 +39,39 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[customer_address] -----PhysicalQuickSort -------PhysicalDistribute ---------PhysicalQuickSort -----------PhysicalProject -------------hashJoin[INNER_JOIN](ws1.ca_county = ws3.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) ---------------PhysicalDistribute +----PhysicalResultSink +------PhysicalQuickSort +--------PhysicalDistribute +----------PhysicalQuickSort +------------PhysicalProject +--------------hashJoin[INNER_JOIN](ws1.ca_county = ws3.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((ws3.d_year = 2000)(ws3.d_qoy = 3)) +----------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------PhysicalProject -------------------filter((ws3.d_year = 2000)(ws3.d_qoy = 3)) ---------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](ss2.ca_county = ss3.ca_county) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((ss3.d_year = 2000)(ss3.d_qoy = 3)) -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------hashJoin[INNER_JOIN](ws1.ca_county = ws2.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) ---------------------hashJoin[INNER_JOIN](ss1.ca_county = ws1.ca_county) -----------------------hashJoin[INNER_JOIN](ss1.ca_county = ss2.ca_county) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((ss1.d_year = 2000)(ss1.d_qoy = 1)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------hashJoin[INNER_JOIN](ss2.ca_county = ss3.ca_county) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((ss3.d_year = 2000)(ss3.d_qoy = 3)) +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------hashJoin[INNER_JOIN](ws1.ca_county = ws2.ca_county)(CASE WHEN (web_sales > 0.00) THEN (cast(web_sales as DECIMALV3(38, 8)) / web_sales) ELSE NULL END > CASE WHEN (store_sales > 0.00) THEN (cast(store_sales as DECIMALV3(38, 8)) / store_sales) ELSE NULL END) +----------------------hashJoin[INNER_JOIN](ss1.ca_county = ws1.ca_county) +------------------------hashJoin[INNER_JOIN](ss1.ca_county = ss2.ca_county) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((ss1.d_year = 2000)(ss1.d_qoy = 1)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((ss2.d_year = 2000)(ss2.d_qoy = 2)) +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((ss2.d_year = 2000)(ss2.d_qoy = 2)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------filter((ws1.d_year = 2000)(ws1.d_qoy = 1)) +------------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((ws1.d_year = 2000)(ws1.d_qoy = 1)) +--------------------------filter((ws2.d_qoy = 2)(ws2.d_year = 2000)) ----------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((ws2.d_qoy = 2)(ws2.d_year = 2000)) ---------------------------PhysicalCteConsumer ( cteId=CTEId#1 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out index 1338d8dc1b1677..2f2c494bb25224 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query32.out @@ -1,26 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_32 -- -PhysicalLimit +PhysicalResultSink --PhysicalLimit -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] +----PhysicalLimit +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(cs_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(cs_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) +--------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 29)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 29)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date >= 1999-01-07)(date_dim.d_date <= 1999-04-07)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date >= 1999-01-07)(date_dim.d_date <= 1999-04-07)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out index 731dc270826421..cea4ec9ccc22be 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query33.out @@ -1,14 +1,14 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_33 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] @@ -31,14 +31,14 @@ PhysicalTopN --------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) ----------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((item.i_category = 'Home')) -------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------PhysicalProject ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] @@ -61,39 +61,40 @@ PhysicalTopN --------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) ----------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((item.i_category = 'Home')) -------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Home')) +--------------------------------------PhysicalOlapScan[item] +----------------PhysicalProject ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute ----------------------hashAgg[LOCAL] ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[item] -----------------------------PhysicalDistribute +--------------------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact_id = item.i_manufact_id) +----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ------------------------------PhysicalProject ---------------------------------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 +--------------------------------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 ---------------------------------------------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] +--------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_moy = 1)(date_dim.d_year = 2002)) +------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((customer_address.ca_gmt_offset = -5.00)) +------------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_category = 'Home')) +----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out index 952cc6ee639d49..6fd519ecc69bae 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query34.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_34 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) -----------PhysicalProject -------------PhysicalOlapScan[customer] -----------PhysicalDistribute -------------filter((dn.cnt <= 20)(dn.cnt >= 15)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) +------------PhysicalProject +--------------PhysicalOlapScan[customer] +------------PhysicalDistribute +--------------filter((dn.cnt <= 20)(dn.cnt >= 15)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28)))d_year IN (1998, 1999, 2000)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((((date_dim.d_dom >= 1) AND (date_dim.d_dom <= 3)) OR ((date_dim.d_dom >= 25) AND (date_dim.d_dom <= 28)))d_year IN (1998, 1999, 2000)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '1001-5000') OR (cast(hd_buy_potential as VARCHAR(*)) = '0-500'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.2)) +----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '1001-5000') OR (cast(hd_buy_potential as VARCHAR(*)) = '0-500'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.2)) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(s_county IN ('Ziebach County', 'Daviess County', 'Walker County', 'Richland County', 'Barrow County', 'Franklin Parish', 'Williamson County', 'Luce County')) -------------------------------PhysicalOlapScan[store] +------------------------------filter(s_county IN ('Ziebach County', 'Daviess County', 'Walker County', 'Richland County', 'Barrow County', 'Franklin Parish', 'Williamson County', 'Luce County')) +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out index 93dac990e56444..1a1d022d75b7b9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query35.out @@ -1,58 +1,59 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_35 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------filter(($c$1 OR $c$2)) -------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) ---------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) -----------------------------------PhysicalDistribute -------------------------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------filter(($c$1 OR $c$2)) +--------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) +----------------------hashJoin[LEFT_SEMI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) ---------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[customer] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute +------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) +----------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[customer] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[customer_address] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_demographics] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalOlapScan[catalog_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) ----------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((date_dim.d_qoy < 4)(date_dim.d_year = 2001)) ---------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out index c5a8b73dc1a685..eed6724c0d7879 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query36.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_36 -- -PhysicalProject ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalRepeat ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) -----------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalProject +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalRepeat +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](item.i_item_sk = store_sales.ss_item_sk) +------------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((d1.d_year = 2002)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((d1.d_year = 2002)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[item] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter(s_state IN ('SD', 'TN', 'GA', 'SC', 'MO', 'AL', 'MI', 'OH')) -------------------------------------PhysicalOlapScan[store] +----------------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter(s_state IN ('SD', 'TN', 'GA', 'SC', 'MO', 'AL', 'MI', 'OH')) +--------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out index a2301a7eec0c38..f9c9f7ec3ea664 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query37.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_37 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[catalog_sales] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) -----------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) -------------------------PhysicalProject ---------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) -----------------------------PhysicalOlapScan[inventory] +--------------------PhysicalOlapScan[catalog_sales] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) +------------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +--------------------------PhysicalProject +----------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) +------------------------------PhysicalOlapScan[inventory] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((item.i_current_price <= 75.00)i_manufact_id IN (707, 747, 856, 1000)(item.i_current_price >= 45.00)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((item.i_current_price <= 75.00)i_manufact_id IN (707, 747, 856, 1000)(item.i_current_price >= 45.00)) -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((date_dim.d_date >= 1999-02-21)(date_dim.d_date <= 1999-04-22)) -----------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_date >= 1999-02-21)(date_dim.d_date <= 1999-04-22)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out index 02fcd034cecb7e..3b8e5596cc2be6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query38.out @@ -1,64 +1,65 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_38 -- -PhysicalLimit +PhysicalResultSink --PhysicalLimit -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------PhysicalIntersect ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] ---------------------------------PhysicalDistribute +----PhysicalLimit +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------PhysicalIntersect +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_sales] ---------------------------------PhysicalDistribute +------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +----------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] ---------------------------------PhysicalDistribute +------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +----------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) ---------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[customer] +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_month_seq <= 1194)(date_dim.d_month_seq >= 1183)) +----------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out index bd6e7a2a90ff73..a35d2c59bb1251 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query39.out @@ -22,17 +22,18 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ------------------PhysicalDistribute --------------------PhysicalProject ----------------------PhysicalOlapScan[warehouse] ---PhysicalQuickSort -----PhysicalDistribute -------PhysicalQuickSort ---------PhysicalProject -----------hashJoin[INNER_JOIN](inv1.w_warehouse_sk = inv2.w_warehouse_sk)(inv1.i_item_sk = inv2.i_item_sk) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((inv1.d_moy = 1)) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((inv2.d_moy = 2)) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalQuickSort +------PhysicalDistribute +--------PhysicalQuickSort +----------PhysicalProject +------------hashJoin[INNER_JOIN](inv1.w_warehouse_sk = inv2.w_warehouse_sk)(inv1.i_item_sk = inv2.i_item_sk) +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((inv1.d_moy = 1)) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((inv2.d_moy = 2)) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out index 1ea2eae2bcf004..540045f8f467fb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query4.out @@ -60,40 +60,41 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter('w' IN ('c', 's', 'w')d_year IN (2000, 1999)) ------------------------PhysicalOlapScan[date_dim] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END) -------------PhysicalProject ---------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) -------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's')) ---------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END) +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id) +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_secyear.customer_id)(CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END > CASE WHEN (year_total > 0.000000) THEN (cast(year_total as DECIMALV3(38, 16)) / year_total) ELSE NULL END) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) +--------------------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_c_firstyear.customer_id) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((t_s_firstyear.year_total > 0.000000)(t_s_firstyear.dyear = 1999)(t_s_firstyear.sale_type = 's')) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c')) +----------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((t_c_firstyear.year_total > 0.000000)(t_c_firstyear.dyear = 1999)(t_c_firstyear.sale_type = 'c')) +------------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000)) --------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.dyear = 2000)) -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((t_c_secyear.sale_type = 'c')(t_c_secyear.dyear = 2000)) ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((t_w_firstyear.dyear = 1999)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((t_w_secyear.sale_type = 'w')(t_w_secyear.dyear = 2000)) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((t_c_secyear.sale_type = 'c')(t_c_secyear.dyear = 2000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_w_firstyear.dyear = 1999)(t_w_firstyear.sale_type = 'w')(t_w_firstyear.year_total > 0.000000)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((t_w_secyear.sale_type = 'w')(t_w_secyear.dyear = 2000)) +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out index 693a3f81360590..6c1818fbfdf8e4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query40.out @@ -1,19 +1,19 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_40 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) -----------------PhysicalProject -------------------PhysicalOlapScan[catalog_returns] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) +------------------PhysicalProject +--------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +----------------------PhysicalProject +------------------------PhysicalOlapScan[catalog_returns] +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) --------------------------PhysicalProject ----------------------------PhysicalOlapScan[catalog_sales] @@ -23,9 +23,9 @@ PhysicalTopN --------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[warehouse] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((date_dim.d_date >= 2001-03-03)(date_dim.d_date <= 2001-05-02)) ---------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_date >= 2001-03-03)(date_dim.d_date <= 2001-05-02)) +------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out index 1d86178c4c2c55..9a5ec3b3811873 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query41.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_41 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact = i1.i_manufact) -----------------PhysicalProject -------------------filter((i1.i_manufact_id >= 748)(i1.i_manufact_id <= 788)) ---------------------PhysicalOlapScan[item] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[LEFT_SEMI_JOIN](item.i_manufact = i1.i_manufact) ------------------PhysicalProject ---------------------filter((item_cnt > 0)) -----------------------hashAgg[GLOBAL] -------------------------PhysicalDistribute ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------filter((((((((((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'gainsboro') OR (cast(i_color as VARCHAR(*)) = 'aquamarine'))) AND ((cast(i_units as VARCHAR(*)) = 'Ounce') OR (cast(i_units as VARCHAR(*)) = 'Dozen'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy'))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'chiffon') OR (cast(i_color as VARCHAR(*)) = 'violet'))) AND ((cast(i_units as VARCHAR(*)) = 'Ton') OR (cast(i_units as VARCHAR(*)) = 'Pound'))) AND ((cast(i_size as VARCHAR(*)) = 'extra large') OR (cast(i_size as VARCHAR(*)) = 'small')))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'blanched') OR (cast(i_color as VARCHAR(*)) = 'tomato'))) AND ((cast(i_units as VARCHAR(*)) = 'Tbl') OR (cast(i_units as VARCHAR(*)) = 'Case'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy')))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'almond') OR (cast(i_color as VARCHAR(*)) = 'lime'))) AND ((cast(i_units as VARCHAR(*)) = 'Box') OR (cast(i_units as VARCHAR(*)) = 'Dram'))) AND ((cast(i_size as VARCHAR(*)) = 'extra large') OR (cast(i_size as VARCHAR(*)) = 'small')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'chartreuse') OR (cast(i_color as VARCHAR(*)) = 'blue'))) AND ((cast(i_units as VARCHAR(*)) = 'Each') OR (cast(i_units as VARCHAR(*)) = 'Oz'))) AND ((cast(i_size as VARCHAR(*)) = 'N/A') OR (cast(i_size as VARCHAR(*)) = 'large')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'tan') OR (cast(i_color as VARCHAR(*)) = 'dodger'))) AND ((cast(i_units as VARCHAR(*)) = 'Bunch') OR (cast(i_units as VARCHAR(*)) = 'Tsp'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'peru') OR (cast(i_color as VARCHAR(*)) = 'saddle'))) AND ((cast(i_units as VARCHAR(*)) = 'Pallet') OR (cast(i_units as VARCHAR(*)) = 'Gram'))) AND ((cast(i_size as VARCHAR(*)) = 'N/A') OR (cast(i_size as VARCHAR(*)) = 'large')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'indian') OR (cast(i_color as VARCHAR(*)) = 'spring'))) AND ((cast(i_units as VARCHAR(*)) = 'Unknown') OR (cast(i_units as VARCHAR(*)) = 'Carton'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy'))))) ---------------------------------PhysicalOlapScan[item] +--------------------filter((i1.i_manufact_id >= 748)(i1.i_manufact_id <= 788)) +----------------------PhysicalOlapScan[item] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((item_cnt > 0)) +------------------------hashAgg[GLOBAL] +--------------------------PhysicalDistribute +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------filter((((((((((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'gainsboro') OR (cast(i_color as VARCHAR(*)) = 'aquamarine'))) AND ((cast(i_units as VARCHAR(*)) = 'Ounce') OR (cast(i_units as VARCHAR(*)) = 'Dozen'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy'))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'chiffon') OR (cast(i_color as VARCHAR(*)) = 'violet'))) AND ((cast(i_units as VARCHAR(*)) = 'Ton') OR (cast(i_units as VARCHAR(*)) = 'Pound'))) AND ((cast(i_size as VARCHAR(*)) = 'extra large') OR (cast(i_size as VARCHAR(*)) = 'small')))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'blanched') OR (cast(i_color as VARCHAR(*)) = 'tomato'))) AND ((cast(i_units as VARCHAR(*)) = 'Tbl') OR (cast(i_units as VARCHAR(*)) = 'Case'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy')))) OR ((((cast(i_category as VARCHAR(*)) = 'Women') AND ((cast(i_color as VARCHAR(*)) = 'almond') OR (cast(i_color as VARCHAR(*)) = 'lime'))) AND ((cast(i_units as VARCHAR(*)) = 'Box') OR (cast(i_units as VARCHAR(*)) = 'Dram'))) AND ((cast(i_size as VARCHAR(*)) = 'extra large') OR (cast(i_size as VARCHAR(*)) = 'small')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'chartreuse') OR (cast(i_color as VARCHAR(*)) = 'blue'))) AND ((cast(i_units as VARCHAR(*)) = 'Each') OR (cast(i_units as VARCHAR(*)) = 'Oz'))) AND ((cast(i_size as VARCHAR(*)) = 'N/A') OR (cast(i_size as VARCHAR(*)) = 'large')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'tan') OR (cast(i_color as VARCHAR(*)) = 'dodger'))) AND ((cast(i_units as VARCHAR(*)) = 'Bunch') OR (cast(i_units as VARCHAR(*)) = 'Tsp'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'peru') OR (cast(i_color as VARCHAR(*)) = 'saddle'))) AND ((cast(i_units as VARCHAR(*)) = 'Pallet') OR (cast(i_units as VARCHAR(*)) = 'Gram'))) AND ((cast(i_size as VARCHAR(*)) = 'N/A') OR (cast(i_size as VARCHAR(*)) = 'large')))) OR ((((cast(i_category as VARCHAR(*)) = 'Men') AND ((cast(i_color as VARCHAR(*)) = 'indian') OR (cast(i_color as VARCHAR(*)) = 'spring'))) AND ((cast(i_units as VARCHAR(*)) = 'Unknown') OR (cast(i_units as VARCHAR(*)) = 'Carton'))) AND ((cast(i_size as VARCHAR(*)) = 'medium') OR (cast(i_size as VARCHAR(*)) = 'economy'))))) +----------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out index f717b0d5783e17..298efbd86325e4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query42.out @@ -1,22 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_42 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) +------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_sales] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((item.i_manager_id = 1)) +--------------------------PhysicalOlapScan[item] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((item.i_manager_id = 1)) -------------------------PhysicalOlapScan[item] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((dt.d_year = 2002)(dt.d_moy = 11)) -----------------------PhysicalOlapScan[date_dim] +----------------------filter((dt.d_year = 2002)(dt.d_moy = 11)) +------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out index f36cfba5ae002e..53978bf7a7e916 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query43.out @@ -1,22 +1,23 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_43 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +--------------------PhysicalProject +----------------------PhysicalOlapScan[store_sales] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------filter((date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((store.s_gmt_offset = -5.00)) -----------------------PhysicalOlapScan[store] +----------------------filter((store.s_gmt_offset = -5.00)) +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out index 5a3c0b08d90a9c..3f3f830e5d3a77 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query44.out @@ -1,17 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_44 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk) -----------PhysicalProject -------------PhysicalOlapScan[item] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](asceding.rnk = descending.rnk) ---------------hashJoin[INNER_JOIN](i2.i_item_sk = descending.item_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[item] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](i1.i_item_sk = asceding.item_sk) +------------PhysicalProject +--------------PhysicalOlapScan[item] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](asceding.rnk = descending.rnk) +----------------hashJoin[INNER_JOIN](i2.i_item_sk = descending.item_sk) +------------------PhysicalProject +--------------------PhysicalOlapScan[item] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((rnk < 11)) +------------------------PhysicalWindow +--------------------------PhysicalQuickSort +----------------------------PhysicalDistribute +------------------------------PhysicalQuickSort +--------------------------------PhysicalPartitionTopN +----------------------------------PhysicalProject +------------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) +--------------------------------------hashAgg[GLOBAL] +----------------------------------------PhysicalDistribute +------------------------------------------hashAgg[LOCAL] +--------------------------------------------PhysicalProject +----------------------------------------------filter((ss1.ss_store_sk = 146)) +------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalAssertNumRows +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------hashAgg[GLOBAL] +------------------------------------------------PhysicalDistribute +--------------------------------------------------hashAgg[LOCAL] +----------------------------------------------------PhysicalProject +------------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) +--------------------------------------------------------PhysicalOlapScan[store_sales] ----------------PhysicalDistribute ------------------PhysicalProject --------------------filter((rnk < 11)) @@ -38,30 +65,4 @@ PhysicalTopN --------------------------------------------------PhysicalProject ----------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) ------------------------------------------------------PhysicalOlapScan[store_sales] ---------------PhysicalDistribute -----------------PhysicalProject -------------------filter((rnk < 11)) ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------PhysicalQuickSort -----------------------------PhysicalPartitionTopN -------------------------------PhysicalProject ---------------------------------NestedLoopJoin[INNER_JOIN](cast(rank_col as DOUBLE) > cast((0.9 * rank_col) as DOUBLE)) -----------------------------------hashAgg[GLOBAL] -------------------------------------PhysicalDistribute ---------------------------------------hashAgg[LOCAL] -----------------------------------------PhysicalProject -------------------------------------------filter((ss1.ss_store_sk = 146)) ---------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalAssertNumRows ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------hashAgg[GLOBAL] ---------------------------------------------PhysicalDistribute -----------------------------------------------hashAgg[LOCAL] -------------------------------------------------PhysicalProject ---------------------------------------------------filter(ss_addr_sk IS NULL(store_sales.ss_store_sk = 146)) -----------------------------------------------------PhysicalOlapScan[store_sales] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out index e84dbdaabe0fb7..83d87e1582a246 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query45.out @@ -1,41 +1,42 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_45 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR $c$1)) -----------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_qoy = 2)(date_dim.d_year = 2000)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------filter((substring(ca_zip, 1, 5) IN ('85669', '86197', '88274', '83405', '86475', '85392', '85460', '80348', '81792') OR $c$1)) +------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) --------------------PhysicalProject -----------------------filter(i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)) -------------------------PhysicalOlapScan[item] +----------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------------PhysicalDistribute +--------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((date_dim.d_qoy = 2)(date_dim.d_year = 2000)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter(i_item_sk IN (2, 3, 5, 7, 11, 13, 17, 19, 23, 29)) +--------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out index bd24b2de19c35c..a21333fa4286af 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query46.out @@ -1,42 +1,43 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_46 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk)( not (ca_city = bought_city)) -----------PhysicalDistribute -------------PhysicalProject ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk)( not (ca_city = bought_city)) +------------PhysicalDistribute +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter(d_dow IN (0, 6)d_year IN (1999, 2000, 2001)) +--------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter(d_dow IN (0, 6)d_year IN (1999, 2000, 2001)) -------------------------------------PhysicalOlapScan[date_dim] +----------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) +------------------------------------PhysicalOlapScan[household_demographics] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(((household_demographics.hd_dep_count = 6) OR (household_demographics.hd_vehicle_count = 0))) -----------------------------------PhysicalOlapScan[household_demographics] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter(s_city IN ('Five Points', 'Centerville', 'Oak Grove', 'Fairview', 'Liberty')) ---------------------------------PhysicalOlapScan[store] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer_address] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk) ---------------PhysicalProject -----------------PhysicalOlapScan[customer_address] ---------------PhysicalDistribute +--------------------------------filter(s_city IN ('Five Points', 'Centerville', 'Oak Grove', 'Fairview', 'Liberty')) +----------------------------------PhysicalOlapScan[store] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer_address] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk) ----------------PhysicalProject -------------------PhysicalOlapScan[customer] +------------------PhysicalOlapScan[customer_address] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out index 7b27c69128a9b9..b705f7a2cbf388 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query47.out @@ -29,23 +29,24 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[store] ---PhysicalProject -----PhysicalTopN -------PhysicalDistribute ---------PhysicalTopN -----------PhysicalProject -------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 2001)(v2.avg_monthly_sales > 0.0000)) +--PhysicalResultSink +----PhysicalProject +------PhysicalTopN +--------PhysicalDistribute +----------PhysicalTopN +------------PhysicalProject +--------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.s_store_name = v1_lead.s_store_name)(v1.s_company_name = v1_lead.s_company_name)(v1.rn = expr_(rn - 1)) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.s_store_name = v1_lag.s_store_name)(v1.s_company_name = v1_lag.s_company_name)(v1.rn = expr_(rn + 1)) +----------------------PhysicalDistribute +------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 2001)(v2.avg_monthly_sales > 0.0000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out index 304b8fbcf04a30..00af1ea47653ed 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query48.out @@ -1,31 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_48 -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -----------PhysicalProject -------------PhysicalOlapScan[store] -----------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('MD', 'MN', 'IA') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('VA', 'IL', 'TX') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('MI', 'WI', 'IN') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))) -----------------PhysicalProject -------------------filter(((ca_state IN ('MD', 'MN', 'IA') OR ca_state IN ('VA', 'IL', 'TX')) OR ca_state IN ('MI', 'WI', 'IN'))(customer_address.ca_country = 'United States')) ---------------------PhysicalOlapScan[customer_address] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)(((((cast(cd_marital_status as VARCHAR(*)) = 'U') AND (cast(cd_education_status as VARCHAR(*)) = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) -------------------------PhysicalProject ---------------------------filter(((((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) -----------------------------PhysicalOlapScan[store_sales] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'U') AND (cast(cd_education_status as VARCHAR(*)) = 'Primary')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'College'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = '2 yr Degree')))) -------------------------------PhysicalOlapScan[customer_demographics] +--------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk)(((ca_state IN ('MD', 'MN', 'IA') AND ((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00))) OR (ca_state IN ('VA', 'IL', 'TX') AND ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00)))) OR (ca_state IN ('MI', 'WI', 'IN') AND ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = store_sales.ss_cdemo_sk)(((((cast(cd_marital_status as VARCHAR(*)) = 'U') AND (cast(cd_education_status as VARCHAR(*)) = 'Primary')) AND ((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'College')) AND ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00)))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = '2 yr Degree')) AND ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) +----------------------PhysicalProject +------------------------filter(((((store_sales.ss_net_profit >= 0.00) AND (store_sales.ss_net_profit <= 2000.00)) OR ((store_sales.ss_net_profit >= 150.00) AND (store_sales.ss_net_profit <= 3000.00))) OR ((store_sales.ss_net_profit >= 50.00) AND (store_sales.ss_net_profit <= 25000.00)))((((store_sales.ss_sales_price >= 100.00) AND (store_sales.ss_sales_price <= 150.00)) OR ((store_sales.ss_sales_price >= 50.00) AND (store_sales.ss_sales_price <= 100.00))) OR ((store_sales.ss_sales_price >= 150.00) AND (store_sales.ss_sales_price <= 200.00)))) +--------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((date_dim.d_year = 1999)) -----------------------------PhysicalOlapScan[date_dim] +--------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'U') AND (cast(cd_education_status as VARCHAR(*)) = 'Primary')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'College'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'D') AND (cast(cd_education_status as VARCHAR(*)) = '2 yr Degree')))) +----------------------------PhysicalOlapScan[customer_demographics] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter(((ca_state IN ('MD', 'MN', 'IA') OR ca_state IN ('VA', 'IL', 'TX')) OR ca_state IN ('MI', 'WI', 'IN'))(customer_address.ca_country = 'United States')) +------------------------PhysicalOlapScan[customer_address] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((date_dim.d_year = 1999)) +----------------------PhysicalOlapScan[date_dim] +------------PhysicalDistribute +--------------PhysicalProject +----------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out index 188d6d3c7b282b..b5dc9c3f7050a1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query49.out @@ -1,86 +1,87 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_49 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[LOCAL] ---------PhysicalUnion -----------PhysicalDistribute -------------PhysicalProject ---------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------PhysicalWindow -------------------PhysicalQuickSort ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------PhysicalQuickSort -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](ws.ws_order_number = wr.wr_order_number)(item = wr.wr_item_sk) -----------------------------------------PhysicalProject -------------------------------------------filter((wr.wr_return_amt > 10000.00)) ---------------------------------------------PhysicalOlapScan[web_returns] -----------------------------------------hashJoin[INNER_JOIN](ws.ws_sold_date_sk = date_dim.d_date_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[LOCAL] +----------PhysicalUnion +------------PhysicalDistribute +--------------PhysicalProject +----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------------PhysicalWindow +--------------------PhysicalQuickSort +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------PhysicalQuickSort +------------------------------PhysicalProject +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](ws.ws_order_number = wr.wr_order_number)(item = wr.wr_item_sk) ------------------------------------------PhysicalProject ---------------------------------------------filter((ws.ws_net_paid > 0.00)(ws.ws_quantity > 0)(ws.ws_net_profit > 1.00)) -----------------------------------------------PhysicalOlapScan[web_sales] -------------------------------------------PhysicalDistribute +--------------------------------------------filter((wr.wr_return_amt > 10000.00)) +----------------------------------------------PhysicalOlapScan[web_returns] +------------------------------------------hashJoin[INNER_JOIN](ws.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------PhysicalOlapScan[date_dim] -----------PhysicalDistribute -------------PhysicalProject ---------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------PhysicalWindow -------------------PhysicalQuickSort ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------PhysicalQuickSort -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](cs.cs_order_number = cr.cr_order_number)(item = cr.cr_item_sk) -----------------------------------------PhysicalProject -------------------------------------------filter((cr.cr_return_amount > 10000.00)) ---------------------------------------------PhysicalOlapScan[catalog_returns] -----------------------------------------hashJoin[INNER_JOIN](cs.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------------------------filter((ws.ws_net_paid > 0.00)(ws.ws_quantity > 0)(ws.ws_net_profit > 1.00)) +------------------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------------PhysicalOlapScan[date_dim] +------------PhysicalDistribute +--------------PhysicalProject +----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------------PhysicalWindow +--------------------PhysicalQuickSort +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------PhysicalQuickSort +------------------------------PhysicalProject +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](cs.cs_order_number = cr.cr_order_number)(item = cr.cr_item_sk) ------------------------------------------PhysicalProject ---------------------------------------------filter((cs.cs_net_paid > 0.00)(cs.cs_quantity > 0)(cs.cs_net_profit > 1.00)) -----------------------------------------------PhysicalOlapScan[catalog_sales] -------------------------------------------PhysicalDistribute +--------------------------------------------filter((cr.cr_return_amount > 10000.00)) +----------------------------------------------PhysicalOlapScan[catalog_returns] +------------------------------------------hashJoin[INNER_JOIN](cs.cs_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------PhysicalOlapScan[date_dim] -----------PhysicalDistribute -------------PhysicalProject ---------------filter(((return_rank <= 10) OR (currency_rank <= 10))) -----------------PhysicalWindow -------------------PhysicalQuickSort ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------PhysicalQuickSort -----------------------------PhysicalProject -------------------------------hashAgg[GLOBAL] ---------------------------------PhysicalDistribute -----------------------------------hashAgg[LOCAL] -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](sts.ss_ticket_number = sr.sr_ticket_number)(item = sr.sr_item_sk) -----------------------------------------PhysicalProject -------------------------------------------filter((sr.sr_return_amt > 10000.00)) ---------------------------------------------PhysicalOlapScan[store_returns] -----------------------------------------hashJoin[INNER_JOIN](sts.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------------------filter((cs.cs_net_paid > 0.00)(cs.cs_quantity > 0)(cs.cs_net_profit > 1.00)) +------------------------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------------PhysicalOlapScan[date_dim] +------------PhysicalDistribute +--------------PhysicalProject +----------------filter(((return_rank <= 10) OR (currency_rank <= 10))) +------------------PhysicalWindow +--------------------PhysicalQuickSort +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------PhysicalQuickSort +------------------------------PhysicalProject +--------------------------------hashAgg[GLOBAL] +----------------------------------PhysicalDistribute +------------------------------------hashAgg[LOCAL] +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](sts.ss_ticket_number = sr.sr_ticket_number)(item = sr.sr_item_sk) ------------------------------------------PhysicalProject ---------------------------------------------filter((sts.ss_quantity > 0)(sts.ss_net_profit > 1.00)(sts.ss_net_paid > 0.00)) -----------------------------------------------PhysicalOlapScan[store_sales] -------------------------------------------PhysicalDistribute +--------------------------------------------filter((sr.sr_return_amt > 10000.00)) +----------------------------------------------PhysicalOlapScan[store_returns] +------------------------------------------hashJoin[INNER_JOIN](sts.ss_sold_date_sk = date_dim.d_date_sk) --------------------------------------------PhysicalProject -----------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) -------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------filter((sts.ss_quantity > 0)(sts.ss_net_profit > 1.00)(sts.ss_net_paid > 0.00)) +------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1999)) +--------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out index eb45d441220307..49a99dc3af856f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query5.out @@ -1,79 +1,80 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_5 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](salesreturns.store_sk = store.s_store_sk) -------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) ---------------------------------PhysicalUnion +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](salesreturns.store_sk = store.s_store_sk) +--------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +----------------------------------PhysicalUnion +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------filter((date_dim.d_date <= 2000-09-02)(date_dim.d_date >= 2000-08-19)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_date <= 2000-09-02)(date_dim.d_date >= 2000-08-19)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](salesreturns.page_sk = catalog_page.cp_catalog_page_sk) -------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) ---------------------------------PhysicalUnion -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](salesreturns.page_sk = catalog_page.cp_catalog_page_sk) +--------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +----------------------------------PhysicalUnion +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_returns] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[catalog_page] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](salesreturns.wsr_web_site_sk = web_site.web_site_sk) -------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) ---------------------------------PhysicalUnion -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalOlapScan[catalog_page] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](salesreturns.wsr_web_site_sk = web_site.web_site_sk) +--------------------------------hashJoin[INNER_JOIN](salesreturns.date_sk = date_dim.d_date_sk) +----------------------------------PhysicalUnion +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = web_sales.ws_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[web_returns] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = web_sales.ws_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_date >= 2000-08-19)(date_dim.d_date <= 2000-09-02)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_site] +------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out index f1338ac40a2f20..e5ee3833007169 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query50.out @@ -1,29 +1,32 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_50 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------PhysicalDistribute --------------------PhysicalProject -----------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) +----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = d1.d_date_sk) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] ---------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_returns] -----------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number)(store_sales.ss_customer_sk = store_returns.sr_customer_sk) ------------------------------PhysicalProject ---------------------------------filter((d2.d_year = 2001)(d2.d_moy = 8)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[store_sales] +------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = d2.d_date_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((d2.d_year = 2001)(d2.d_moy = 8)) +--------------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out index 66c68bed08495c..8ba49dc8d60bdd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query51.out @@ -1,42 +1,43 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_51 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------filter((web_cumulative > store_cumulative)) ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[FULL_OUTER_JOIN](web.item_sk = store.item_sk)(web.d_date = store.d_date) -------------------PhysicalProject ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] -------------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------filter((web_cumulative > store_cumulative)) +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[FULL_OUTER_JOIN](web.item_sk = store.item_sk)(web.d_date = store.d_date) +--------------------PhysicalProject +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) -------------------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------PhysicalWindow -----------------------PhysicalQuickSort -------------------------PhysicalDistribute ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] -------------------------------------PhysicalDistribute +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_month_seq <= 1227)(date_dim.d_month_seq >= 1216)) +--------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------PhysicalWindow +------------------------PhysicalQuickSort +--------------------------PhysicalDistribute +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) -------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_month_seq >= 1216)(date_dim.d_month_seq <= 1227)) +--------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out index bf191b847cbd5c..4bf2ceed3dcbef 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query52.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_52 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](dt.d_date_sk = store_sales.ss_sold_date_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((dt.d_moy = 12)(dt.d_year = 2002)) -------------------------PhysicalOlapScan[date_dim] +------------------------filter((dt.d_moy = 12)(dt.d_year = 2002)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out index 0e916de7170f7e..87e4abe7797004 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query53.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_53 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter((CASE WHEN (avg_quarterly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_quarterly_sales as DOUBLE))) / cast(avg_quarterly_sales as DOUBLE)) ELSE NULL END > 0.1)) -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter((CASE WHEN (avg_quarterly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_quarterly_sales as DOUBLE))) / cast(avg_quarterly_sales as DOUBLE)) ELSE NULL END > 0.1)) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) +------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) -----------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +--------------------------------------filter(d_month_seq IN (1200, 1201, 1202, 1203, 1204, 1205, 1206, 1207, 1208, 1209, 1210, 1211)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out index e6002e5a2d6c48..70a48fdb34e108 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query54.out @@ -1,82 +1,83 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_54 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer_address.ca_county = store.s_county)(customer_address.ca_state = store.s_state) ---------------------------PhysicalProject -----------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) -------------------------------PhysicalProject ---------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](my_customers.c_customer_sk = store_sales.ss_customer_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] -----------------------------------------PhysicalDistribute -------------------------------------------hashJoin[INNER_JOIN](my_customers.c_current_addr_sk = customer_address.ca_address_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[customer_address] ---------------------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](customer_address.ca_county = store.s_county)(customer_address.ca_state = store.s_state) +----------------------------PhysicalProject +------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) <= (d_month_seq + 3)) +--------------------------------PhysicalProject +----------------------------------NestedLoopJoin[INNER_JOIN](cast(d_month_seq as BIGINT) >= (d_month_seq + 1)) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](my_customers.c_customer_sk = store_sales.ss_customer_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------hashJoin[INNER_JOIN](my_customers.c_current_addr_sk = customer_address.ca_address_sk) ----------------------------------------------PhysicalProject -------------------------------------------------hashAgg[GLOBAL] ---------------------------------------------------PhysicalDistribute -----------------------------------------------------hashAgg[LOCAL] -------------------------------------------------------PhysicalProject ---------------------------------------------------------hashJoin[INNER_JOIN](customer.c_customer_sk = cs_or_ws_sales.customer_sk) -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[customer] -----------------------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------hashAgg[GLOBAL] +----------------------------------------------------PhysicalDistribute +------------------------------------------------------hashAgg[LOCAL] +--------------------------------------------------------PhysicalProject +----------------------------------------------------------hashJoin[INNER_JOIN](customer.c_customer_sk = cs_or_ws_sales.customer_sk) ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk) -----------------------------------------------------------------PhysicalProject -------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.item_sk = item.i_item_sk) ---------------------------------------------------------------------PhysicalUnion -----------------------------------------------------------------------PhysicalDistribute -------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------------------------------------PhysicalOlapScan[customer] +------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.sold_date_sk = date_dim.d_date_sk) +------------------------------------------------------------------PhysicalProject +--------------------------------------------------------------------hashJoin[INNER_JOIN](cs_or_ws_sales.item_sk = item.i_item_sk) +----------------------------------------------------------------------PhysicalUnion +------------------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------------PhysicalOlapScan[web_sales] ----------------------------------------------------------------------PhysicalDistribute ------------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------------PhysicalOlapScan[web_sales] ---------------------------------------------------------------------PhysicalDistribute -----------------------------------------------------------------------PhysicalProject -------------------------------------------------------------------------filter((cast(i_class as VARCHAR(*)) = 'maternity')(cast(i_category as VARCHAR(*)) = 'Women')) ---------------------------------------------------------------------------PhysicalOlapScan[item] -----------------------------------------------------------------PhysicalDistribute -------------------------------------------------------------------PhysicalProject ---------------------------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) -----------------------------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalAssertNumRows ---------------------------------------PhysicalDistribute -----------------------------------------hashAgg[GLOBAL] -------------------------------------------PhysicalDistribute ---------------------------------------------hashAgg[LOCAL] -----------------------------------------------PhysicalProject -------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) ---------------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalAssertNumRows -----------------------------------PhysicalDistribute -------------------------------------hashAgg[GLOBAL] +--------------------------------------------------------------------------filter((cast(i_class as VARCHAR(*)) = 'maternity')(cast(i_category as VARCHAR(*)) = 'Women')) +----------------------------------------------------------------------------PhysicalOlapScan[item] +------------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------------PhysicalProject +----------------------------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +------------------------------------------------------------------------PhysicalOlapScan[date_dim] --------------------------------------PhysicalDistribute -----------------------------------------hashAgg[LOCAL] -------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) -----------------------------------------------PhysicalOlapScan[date_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalAssertNumRows +----------------------------------------PhysicalDistribute +------------------------------------------hashAgg[GLOBAL] +--------------------------------------------PhysicalDistribute +----------------------------------------------hashAgg[LOCAL] +------------------------------------------------PhysicalProject +--------------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +----------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute +--------------------------------------hashAgg[GLOBAL] +----------------------------------------PhysicalDistribute +------------------------------------------hashAgg[LOCAL] +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_year = 1998)(date_dim.d_moy = 5)) +------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out index f4e966c695017e..90db0b7ca05f11 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query55.out @@ -1,23 +1,24 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_55 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((item.i_manager_id = 100)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 100)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 2000)) -------------------------PhysicalOlapScan[date_dim] +------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 2000)) +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out index a7efe6f1d67a0c..b2821d91e97bb3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query56.out @@ -1,93 +1,94 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_56 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter((customer_address.ca_gmt_offset = -6.00)) ---------------------------------PhysicalOlapScan[customer_address] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute +--------------------------------filter((customer_address.ca_gmt_offset = -6.00)) +----------------------------------PhysicalOlapScan[customer_address] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +------------------------------------------PhysicalOlapScan[item] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((customer_address.ca_gmt_offset = -6.00)) +----------------------------------PhysicalOlapScan[customer_address] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) ----------------------------PhysicalProject ------------------------------filter((customer_address.ca_gmt_offset = -6.00)) --------------------------------PhysicalOlapScan[customer_address] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalProject -----------------------------filter((customer_address.ca_gmt_offset = -6.00)) -------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalDistribute +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 2)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) ------------------------------------PhysicalProject ---------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) -----------------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(i_color IN ('powder', 'green', 'cyan')) +------------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out index 0e13df083b4ceb..d3d96f0926e446 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query57.out @@ -29,23 +29,24 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[call_center] ---PhysicalProject -----PhysicalTopN -------PhysicalDistribute ---------PhysicalTopN -----------PhysicalProject -------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1)) ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1)) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 1999)(v2.avg_monthly_sales > 0.0000)) +--PhysicalResultSink +----PhysicalProject +------PhysicalTopN +--------PhysicalDistribute +----------PhysicalTopN +------------PhysicalProject +--------------hashJoin[INNER_JOIN](v1.i_category = v1_lead.i_category)(v1.i_brand = v1_lead.i_brand)(v1.cc_name = v1_lead.cc_name)(v1.rn = expr_(rn - 1)) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](v1.i_category = v1_lag.i_category)(v1.i_brand = v1_lag.i_brand)(v1.cc_name = v1_lag.cc_name)(v1.rn = expr_(rn + 1)) +----------------------PhysicalDistribute +------------------------PhysicalProject --------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)(v2.d_year = 1999)(v2.avg_monthly_sales > 0.0000)) +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out index 7ffaac876a79d8..e5fd32f39b6626 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query58.out @@ -1,39 +1,11 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_58 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](ss_items.item_id = cs_items.item_id)(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE)) -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalAssertNumRows ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_date = 2001-03-24)) ---------------------------------------PhysicalOlapScan[date_dim] -----------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](ss_items.item_id = cs_items.item_id)(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) >= cast((0.9 * cs_item_rev) as DOUBLE))(cast(cs_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * cs_item_rev) as DOUBLE)) ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute @@ -41,10 +13,10 @@ PhysicalTopN --------------------PhysicalProject ----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] @@ -62,32 +34,61 @@ PhysicalTopN ------------------------------------PhysicalProject --------------------------------------filter((date_dim.d_date = 2001-03-24)) ----------------------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] +------------hashJoin[INNER_JOIN](ss_items.item_id = ws_items.item_id)(cast(ws_item_rev as DOUBLE) >= cast((0.9 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) >= cast((0.9 * ws_item_rev) as DOUBLE))(cast(ws_item_rev as DOUBLE) <= cast((1.1 * ss_item_rev) as DOUBLE))(cast(ss_item_rev as DOUBLE) <= cast((1.1 * ws_item_rev) as DOUBLE)) +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_date = 2001-03-24)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute ---------------------------------PhysicalAssertNumRows -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_date = 2001-03-24)) -----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalAssertNumRows +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_date = 2001-03-24)) +------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out index 89bed6d2465ef1..5b0559251ee462 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query59.out @@ -13,44 +13,38 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject --------------------PhysicalOlapScan[date_dim] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) -------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) ---------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[store] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) -------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](y.s_store_id1 = x.s_store_id2)(expr_cast(d_week_seq1 as BIGINT) = expr_(d_week_seq2 - 52)) +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +--------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq2) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((d.d_month_seq <= 1219)(d.d_month_seq >= 1208)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------------PhysicalOlapScan[store] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) +--------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((d.d_month_seq <= 1207)(d.d_month_seq >= 1196)) +----------------------------PhysicalOlapScan[date_dim] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](wss.ss_store_sk = store.s_store_sk) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](d.d_week_seq = d_week_seq1) -------------------------------PhysicalProject ---------------------------------CteConsumer[cteId= ( CTEId#4=] ) -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((d.d_month_seq <= 1207)(d.d_month_seq >= 1196)) -------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out index aa6c71623d648c..fff91e211ebb57 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query6.out @@ -1,21 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_6 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------filter((cnt >= 10)) ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[LEFT_SEMI_JOIN](j.i_category = i.i_category)(cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(i_current_price))) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](c.c_customer_sk = s.ss_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](s.ss_item_sk = i.i_item_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------filter((cnt >= 10)) +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[LEFT_SEMI_JOIN](j.i_category = i.i_category)(cast(i_current_price as DECIMALV3(38, 5)) > (1.2 * avg(i_current_price))) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](c.c_customer_sk = s.ss_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](s.ss_item_sk = i.i_item_sk) +------------------------------PhysicalDistribute --------------------------------hashJoin[INNER_JOIN](s.ss_sold_date_sk = d.d_date_sk) ----------------------------------PhysicalProject ------------------------------------PhysicalOlapScan[store_sales] @@ -32,22 +32,22 @@ PhysicalTopN --------------------------------------------------PhysicalProject ----------------------------------------------------filter((date_dim.d_year = 2002)(date_dim.d_moy = 3)) ------------------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](a.ca_address_sk = c.c_current_addr_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[customer_address] -------------------PhysicalDistribute ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[item] +----------------------------hashJoin[INNER_JOIN](a.ca_address_sk = c.c_current_addr_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalDistribute +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out index 8538ed1f7b5aef..c09adeeea2fcb1 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query60.out @@ -1,94 +1,95 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_60 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +----------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute +------------------------------------filter((item.i_category = 'Children')) +--------------------------------------PhysicalOlapScan[item] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_moy = 8)(date_dim.d_year = 2000)) +------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +----------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------------PhysicalOlapScan[customer_address] ----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalDistribute +------------------------------------filter((item.i_category = 'Children')) +--------------------------------------PhysicalOlapScan[item] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------hashJoin[INNER_JOIN](web_sales.ws_bill_addr_sk = customer_address.ca_address_sk) +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((customer_address.ca_gmt_offset = -7.00)) -----------------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute -----------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_year = 2000)(date_dim.d_moy = 8)) +------------------------------------------PhysicalOlapScan[date_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((item.i_category = 'Children')) -------------------------------------PhysicalOlapScan[item] +----------------------------------filter((customer_address.ca_gmt_offset = -7.00)) +------------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------hashJoin[LEFT_SEMI_JOIN](item.i_item_id = item.i_item_id) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[item] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((item.i_category = 'Children')) +--------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out index 072c3b8f853bcd..bc1155555ed9b3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query61.out @@ -1,48 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_61 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----PhysicalProject -------NestedLoopJoin[CROSS_JOIN] ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------PhysicalProject ---------------------filter((cast(i_category as VARCHAR(*)) = 'Jewelry')) -----------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) -------------------------PhysicalProject ---------------------------filter((((cast(p_channel_dmail as VARCHAR(*)) = 'Y') OR (cast(p_channel_email as VARCHAR(*)) = 'Y')) OR (cast(p_channel_tv as VARCHAR(*)) = 'Y'))) -----------------------------PhysicalOlapScan[promotion] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -----------------------------------PhysicalProject -------------------------------------filter((customer_address.ca_gmt_offset = -7.00)) ---------------------------------------PhysicalOlapScan[customer_address] -----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((store.s_gmt_offset = -7.00)) ---------------------------------------------------PhysicalOlapScan[store] ---------PhysicalDistribute +----PhysicalTopN +------PhysicalProject +--------NestedLoopJoin[CROSS_JOIN] ----------hashAgg[GLOBAL] ------------PhysicalDistribute --------------hashAgg[LOCAL] @@ -58,21 +20,60 @@ PhysicalTopN ----------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[customer] ------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] -----------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) ------------------------------------PhysicalProject ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------------filter((((cast(p_channel_dmail as VARCHAR(*)) = 'Y') OR (cast(p_channel_email as VARCHAR(*)) = 'Y')) OR (cast(p_channel_tv as VARCHAR(*)) = 'Y'))) +----------------------------------------PhysicalOlapScan[promotion] +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) +--------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((store.s_gmt_offset = -7.00)) -----------------------------------------------PhysicalOlapScan[store] +--------------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((store.s_gmt_offset = -7.00)) +----------------------------------------------------PhysicalOlapScan[store] +----------PhysicalDistribute +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) +----------------------PhysicalProject +------------------------filter((customer_address.ca_gmt_offset = -7.00)) +--------------------------PhysicalOlapScan[customer_address] +----------------------PhysicalDistribute +------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------PhysicalProject +----------------------------filter((cast(i_category as VARCHAR(*)) = 'Jewelry')) +------------------------------PhysicalOlapScan[item] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 1999)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((store.s_gmt_offset = -7.00)) +------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out index e66a13f8feb250..ec78b77b89e5f3 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query62.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_62 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) -----------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------hashJoin[INNER_JOIN](web_sales.ws_ship_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] +----------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) +------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_date_sk = date_dim.d_date_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq >= 1194)(date_dim.d_month_seq <= 1205)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq >= 1194)(date_dim.d_month_seq <= 1205)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalOlapScan[web_site] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------PhysicalOlapScan[web_site] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out index 906693680efaad..eea2cefd61a2fa 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query63.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_63 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)) -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter((CASE WHEN (avg_monthly_sales > 0.0000) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) +------------------------------------------PhysicalOlapScan[item] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((((i_category IN ('Books', 'Children', 'Electronics') AND i_class IN ('personal', 'portable', 'reference', 'self-help')) AND i_brand IN ('scholaramalgamalg #14', 'scholaramalgamalg #7', 'exportiunivamalg #9', 'scholaramalgamalg #9')) OR ((i_category IN ('Women', 'Music', 'Men') AND i_class IN ('accessories', 'classical', 'fragrances', 'pants')) AND i_brand IN ('amalgimporto #1', 'edu packscholar #1', 'exportiimporto #1', 'importoamalg #1')))) -----------------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) ---------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store] +--------------------------------------filter(d_month_seq IN (1181, 1182, 1183, 1184, 1185, 1186, 1187, 1188, 1189, 1190, 1191, 1192)) +----------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out index ba6a5ed5045e6d..2d74160159d40d 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query65.out @@ -5,40 +5,42 @@ PhysicalResultSink ----PhysicalDistribute ------PhysicalTopN --------PhysicalProject -----------hashJoin[INNER_JOIN](store.s_store_sk = sc.ss_store_sk) -------------hashJoin[INNER_JOIN](item.i_item_sk = sc.ss_item_sk) ---------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](sb.ss_store_sk = sc.ss_store_sk)(cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE)) -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1232)(date_dim.d_month_seq >= 1221)) -----------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute +----------hashJoin[INNER_JOIN](item.i_item_sk = sc.ss_item_sk) +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store.s_store_sk = sc.ss_store_sk) +------------------hashJoin[INNER_JOIN](sb.ss_store_sk = sc.ss_store_sk)(cast(revenue as DOUBLE) <= cast((0.1 * ave) as DOUBLE)) --------------------hashAgg[GLOBAL] ----------------------PhysicalDistribute ------------------------hashAgg[LOCAL] --------------------------PhysicalProject -----------------------------hashAgg[GLOBAL] +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[store_sales] ---------------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_month_seq <= 1232)(date_dim.d_month_seq >= 1221)) +------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_month_seq >= 1221)(date_dim.d_month_seq <= 1232)) ---------------------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalOlapScan[item] +------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_month_seq >= 1221)(date_dim.d_month_seq <= 1232)) +----------------------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[store] ------------PhysicalDistribute --------------PhysicalProject -----------------PhysicalOlapScan[store] \ No newline at end of file +----------------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..d83d9cc0f0da0c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query66.out @@ -1,64 +1,65 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_66 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_sales.ws_warehouse_sk = warehouse.w_warehouse_sk) ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_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] +------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) +--------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_ship_mode_sk = ship_mode.sm_ship_mode_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------PhysicalOlapScan[ship_mode] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------PhysicalOlapScan[ship_mode] +--------------------------------------filter((date_dim.d_year = 1998)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] +------------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) +--------------------------------------PhysicalOlapScan[time_dim] ----------------------------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) +--------------------------------PhysicalOlapScan[warehouse] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_time_sk = time_dim.t_time_sk) +--------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) +------------------------------------------PhysicalOlapScan[ship_mode] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter(sm_carrier IN ('GREAT EASTERN', 'LATVIAN')) -----------------------------------------PhysicalOlapScan[ship_mode] +--------------------------------------filter((date_dim.d_year = 1998)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((date_dim.d_year = 1998)) ---------------------------------------PhysicalOlapScan[date_dim] +------------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) +--------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((cast(t_time as BIGINT) <= 77621)(cast(t_time as BIGINT) >= 48821)) -----------------------------------PhysicalOlapScan[time_dim] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[warehouse] +--------------------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out index b2c9bc02f78faa..b9707e1b5a25e9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query67.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_67 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------filter((rk <= 100)) ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------PhysicalPartitionTopN -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalRepeat ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------filter((rk <= 100)) +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------PhysicalPartitionTopN +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalRepeat +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((date_dim.d_month_seq >= 1206)(date_dim.d_month_seq <= 1217)) +------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_month_seq >= 1206)(date_dim.d_month_seq <= 1217)) -----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalOlapScan[item] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[item] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out index b8d7713b309b72..9a98918d915aa5 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query68.out @@ -1,42 +1,44 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_68 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk)( not (ca_city = bought_city)) -----------PhysicalProject -------------PhysicalOlapScan[customer_address] -----------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](customer.c_current_addr_sk = current_addr.ca_address_sk)( not (ca_city = bought_city)) ------------PhysicalProject ---------------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_address] ---------------------------PhysicalDistribute +--------------PhysicalOlapScan[customer_address] +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](dn.ss_customer_sk = customer.c_customer_sk) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_addr_sk = customer_address.ca_address_sk) ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] -------------------------------------PhysicalDistribute ---------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_dom >= 1)d_year IN (1998, 1999, 2000)(date_dim.d_dom <= 2)) -------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[customer_address] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------------------PhysicalProject +------------------------------------------PhysicalOlapScan[store_sales] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_dom >= 1)d_year IN (1998, 1999, 2000)(date_dim.d_dom <= 2)) +----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter(s_city IN ('Pleasant Hill', 'Five Points')) +--------------------------------------------PhysicalOlapScan[store] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter(s_city IN ('Pleasant Hill', 'Five Points')) -----------------------------------------PhysicalOlapScan[store] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) ---------------------------------------PhysicalOlapScan[household_demographics] +--------------------------------------filter(((household_demographics.hd_dep_count = 8) OR (household_demographics.hd_vehicle_count = -1))) +----------------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out index e01ec61085cb5c..2f5465dccc0ef0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query69.out @@ -1,55 +1,56 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_69 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[RIGHT_SEMI_JOIN](c.c_customer_sk = store_sales.ss_customer_sk) +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[store_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[store_sales] +----------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = catalog_sales.cs_ship_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2000)(date_dim.d_moy <= 3)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer_demographics] ---------------------------PhysicalDistribute -----------------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +--------------------------------PhysicalOlapScan[catalog_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------------------filter((date_dim.d_moy >= 1)(date_dim.d_year = 2000)(date_dim.d_moy <= 3)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = c.c_current_cdemo_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer_demographics] +----------------------------PhysicalDistribute +------------------------------hashJoin[RIGHT_ANTI_JOIN](c.c_customer_sk = web_sales.ws_bill_customer_sk) +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) +--------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalOlapScan[customer] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_moy <= 3)(date_dim.d_moy >= 1)(date_dim.d_year = 2000)) -------------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](c.c_current_addr_sk = ca.ca_address_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter(ca_state IN ('TX', 'VA', 'MI')) -----------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------filter(ca_state IN ('TX', 'VA', 'MI')) +------------------------------------------PhysicalOlapScan[customer_address] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out index 69cc851c1d3eb9..574836334751d6 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query7.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_7 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_cdemo_sk = customer_demographics.cd_demo_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')(cast(cd_education_status as VARCHAR(*)) = 'College')(cast(cd_gender as VARCHAR(*)) = 'F')) +----------------------------------PhysicalOlapScan[customer_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')(cast(cd_education_status as VARCHAR(*)) = 'College')(cast(cd_gender as VARCHAR(*)) = 'F')) ---------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2001)) -------------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) ---------------------------PhysicalOlapScan[promotion] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[item] +------------------------------filter((date_dim.d_year = 2001)) +--------------------------------PhysicalOlapScan[date_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(((cast(p_channel_email as VARCHAR(*)) = 'N') OR (cast(p_channel_event as VARCHAR(*)) = 'N'))) +----------------------------PhysicalOlapScan[promotion] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out index b5255185bd4f80..f85e1f6fbfb220 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query70.out @@ -1,50 +1,51 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_70 -- -PhysicalProject ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalRepeat ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) -------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store_sales] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((d1.d_month_seq <= 1224)(d1.d_month_seq >= 1213)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------hashJoin[LEFT_SEMI_JOIN](store.s_state = tmp1.s_state) +PhysicalResultSink +--PhysicalProject +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalRepeat +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +--------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = store_sales.ss_sold_date_sk) ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store] +------------------------------------PhysicalOlapScan[store_sales] ----------------------------------PhysicalDistribute ------------------------------------PhysicalProject ---------------------------------------filter((ranking <= 5)) -----------------------------------------PhysicalWindow -------------------------------------------PhysicalQuickSort ---------------------------------------------PhysicalPartitionTopN -----------------------------------------------hashAgg[GLOBAL] -------------------------------------------------PhysicalDistribute ---------------------------------------------------hashAgg[LOCAL] -----------------------------------------------------PhysicalProject -------------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) ---------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------filter((d1.d_month_seq <= 1224)(d1.d_month_seq >= 1213)) +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[LEFT_SEMI_JOIN](store.s_state = tmp1.s_state) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((ranking <= 5)) +------------------------------------------PhysicalWindow +--------------------------------------------PhysicalQuickSort +----------------------------------------------PhysicalPartitionTopN +------------------------------------------------hashAgg[GLOBAL] +--------------------------------------------------PhysicalDistribute +----------------------------------------------------hashAgg[LOCAL] +------------------------------------------------------PhysicalProject +--------------------------------------------------------hashJoin[INNER_JOIN](store.s_store_sk = store_sales.ss_store_sk) +----------------------------------------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) +------------------------------------------------------------PhysicalProject +--------------------------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------------------------PhysicalDistribute +--------------------------------------------------------------PhysicalProject +----------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224)) +------------------------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------------------------PhysicalDistribute ------------------------------------------------------------PhysicalProject ---------------------------------------------------------------filter((date_dim.d_month_seq >= 1213)(date_dim.d_month_seq <= 1224)) -----------------------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------------------------PhysicalDistribute -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[store] +--------------------------------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out index 650c95b66811dd..0347eb2b3737b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query71.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_71 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](tmp.time_sk = time_dim.t_time_sk) -------------------hashJoin[INNER_JOIN](tmp.sold_item_sk = item.i_item_sk) ---------------------PhysicalUnion -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](tmp.time_sk = time_dim.t_time_sk) +--------------------hashJoin[INNER_JOIN](tmp.sold_item_sk = item.i_item_sk) +----------------------PhysicalUnion +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = catalog_sales.cs_sold_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = store_sales.ss_sold_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_moy = 12)(date_dim.d_year = 1998)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------filter((item.i_manager_id = 1)) +----------------------------PhysicalOlapScan[item] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((item.i_manager_id = 1)) ---------------------------PhysicalOlapScan[item] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter(((cast(t_meal_time as VARCHAR(*)) = 'breakfast') OR (cast(t_meal_time as VARCHAR(*)) = 'dinner'))) -------------------------PhysicalOlapScan[time_dim] +------------------------filter(((cast(t_meal_time as VARCHAR(*)) = 'breakfast') OR (cast(t_meal_time as VARCHAR(*)) = 'dinner'))) +--------------------------PhysicalOlapScan[time_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out index 22010cbb8921da..b3747e4573d7bb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query72.out @@ -19,44 +19,44 @@ PhysicalResultSink ----------------------------PhysicalProject ------------------------------hashJoin[INNER_JOIN](item.i_item_sk = catalog_sales.cs_item_sk) --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = d2.d_date_sk)(d1.d_week_seq = d2.d_week_seq) ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = inventory.inv_item_sk)(inventory.inv_quantity_on_hand < catalog_sales.cs_quantity) +----------------------------------hashJoin[INNER_JOIN](inventory.inv_date_sk = d2.d_date_sk)(d1.d_week_seq = d2.d_week_seq) +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = inventory.inv_item_sk)(inventory.inv_quantity_on_hand < catalog_sales.cs_quantity) +----------------------------------------PhysicalDistribute ------------------------------------------PhysicalOlapScan[inventory] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = d3.d_date_sk)(d3.d_date > cast((cast(d_date as BIGINT) + 5) as DATEV2)) -------------------------------------------------PhysicalDistribute ---------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) -----------------------------------------------------PhysicalDistribute -------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d1.d_date_sk) ---------------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk) -----------------------------------------------------------PhysicalProject -------------------------------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------------------------------PhysicalDistribute -------------------------------------------------------------PhysicalProject ---------------------------------------------------------------filter((cast(hd_buy_potential as VARCHAR(*)) = '501-1000')) -----------------------------------------------------------------PhysicalOlapScan[household_demographics] ---------------------------------------------------------PhysicalDistribute -----------------------------------------------------------PhysicalProject -------------------------------------------------------------filter((d1.d_year = 2002)) ---------------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = d3.d_date_sk)(d3.d_date > cast((cast(d_date as BIGINT) + 5) as DATEV2)) +----------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = d1.d_date_sk) +------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_cdemo_sk = customer_demographics.cd_demo_sk) +--------------------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------------------------------PhysicalProject +------------------------------------------------------PhysicalOlapScan[catalog_sales] ----------------------------------------------------PhysicalDistribute ------------------------------------------------------PhysicalProject ---------------------------------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')) -----------------------------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------------------------------filter((cast(hd_buy_potential as VARCHAR(*)) = '501-1000')) +----------------------------------------------------------PhysicalOlapScan[household_demographics] +--------------------------------------------------PhysicalDistribute +----------------------------------------------------PhysicalProject +------------------------------------------------------filter((cast(cd_marital_status as VARCHAR(*)) = 'W')) +--------------------------------------------------------PhysicalOlapScan[customer_demographics] ------------------------------------------------PhysicalDistribute --------------------------------------------------PhysicalProject -----------------------------------------------------PhysicalOlapScan[date_dim] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------------filter((d1.d_year = 2002)) +------------------------------------------------------PhysicalOlapScan[date_dim] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------PhysicalOlapScan[date_dim] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ----------------------------PhysicalDistribute ------------------------------PhysicalProject --------------------------------PhysicalOlapScan[warehouse] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[promotion] \ No newline at end of file +----------------------------PhysicalOlapScan[promotion] + diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out index 422345128c4877..61342aa658f34b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query73.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_73 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[INNER_JOIN](dj.ss_customer_sk = customer.c_customer_sk) -----------PhysicalProject -------------PhysicalOlapScan[customer] -----------PhysicalDistribute -------------filter((dj.cnt >= 1)(dj.cnt <= 5)) ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[INNER_JOIN](dj.ss_customer_sk = customer.c_customer_sk) +------------PhysicalProject +--------------PhysicalOlapScan[customer] +------------PhysicalDistribute +--------------filter((dj.cnt >= 1)(dj.cnt <= 5)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_dom >= 1)d_year IN (2000, 2001, 2002)(date_dim.d_dom <= 2)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_dom >= 1)d_year IN (2000, 2001, 2002)(date_dim.d_dom <= 2)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '501-1000') OR (cast(hd_buy_potential as VARCHAR(*)) = 'Unknown'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.0)) +----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(((cast(hd_buy_potential as VARCHAR(*)) = '501-1000') OR (cast(hd_buy_potential as VARCHAR(*)) = 'Unknown'))(household_demographics.hd_vehicle_count > 0)(CASE WHEN (hd_vehicle_count > 0) THEN (cast(hd_dep_count as DOUBLE) / cast(hd_vehicle_count as DOUBLE)) ELSE NULL END > 1.0)) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(s_county IN ('Fairfield County', 'Walker County', 'Daviess County', 'Barrow County')) -------------------------------PhysicalOlapScan[store] +------------------------------filter(s_county IN ('Fairfield County', 'Walker County', 'Daviess County', 'Barrow County')) +--------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out index afceaaa1cd27fe..7e915785b8bab2 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query74.out @@ -40,35 +40,28 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------PhysicalProject ----------------------filter('w' IN ('s', 'w')) ------------------------PhysicalOlapScan[customer] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END > CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w')) -------------------CteConsumer[cteId= ( CTEId#4=] ) -------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id) ---------------PhysicalDistribute -----------------PhysicalProject -------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) ---------------------CteConsumer[cteId= ( CTEId#4=] ) ---------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.0)) -----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_firstyear.customer_id)(CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END > CASE WHEN (year_total > 0.0) THEN (year_total / year_total) ELSE NULL END) +--------------hashJoin[INNER_JOIN](t_s_firstyear.customer_id = t_w_secyear.customer_id) +----------------hashJoin[INNER_JOIN](t_s_secyear.customer_id = t_s_firstyear.customer_id) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_s_firstyear.year = 1999)(t_s_firstyear.sale_type = 's')(t_s_firstyear.year_total > 0.0)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000)) +------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ----------------PhysicalDistribute ------------------PhysicalProject ---------------------filter((t_s_secyear.sale_type = 's')(t_s_secyear.year = 2000)) +--------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) ----------------------PhysicalCteConsumer ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject -------------------filter((t_w_secyear.year = 2000)(t_w_secyear.sale_type = 'w')) +------------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w')) --------------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------PhysicalProject -----------------filter((t_w_firstyear.year = 1999)(t_w_firstyear.year_total > 0.0)(t_w_firstyear.sale_type = 'w')) -------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out index 09a72797851f63..50de18777c8ebd 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query75.out @@ -61,15 +61,16 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------------------PhysicalProject ----------------------------filter(d_year IN (1998, 1999)) ------------------------------PhysicalOlapScan[date_dim] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[INNER_JOIN](curr_yr.i_brand_id = prev_yr.i_brand_id)(curr_yr.i_class_id = prev_yr.i_class_id)(curr_yr.i_category_id = prev_yr.i_category_id)(curr_yr.i_manufact_id = prev_yr.i_manufact_id)((cast(cast(sales_cnt as DECIMALV3(17, 2)) as DECIMALV3(23, 8)) / cast(sales_cnt as DECIMALV3(17, 2))) < 0.900000) -------------PhysicalDistribute ---------------filter((curr_yr.d_year = 1999)) -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) -------------PhysicalDistribute ---------------filter((prev_yr.d_year = 1998)) -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[INNER_JOIN](curr_yr.i_brand_id = prev_yr.i_brand_id)(curr_yr.i_class_id = prev_yr.i_class_id)(curr_yr.i_category_id = prev_yr.i_category_id)(curr_yr.i_manufact_id = prev_yr.i_manufact_id)((cast(cast(sales_cnt as DECIMALV3(17, 2)) as DECIMALV3(23, 8)) / cast(sales_cnt as DECIMALV3(17, 2))) < 0.900000) +--------------PhysicalDistribute +----------------filter((curr_yr.d_year = 1999)) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +--------------PhysicalDistribute +----------------filter((prev_yr.d_year = 1998)) +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out index 809d43e7b69bd5..45631094874884 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query76.out @@ -1,49 +1,50 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_76 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalUnion ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------filter(ss_hdemo_sk IS NULL) ---------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalUnion +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +------------------------PhysicalProject +--------------------------filter(ss_hdemo_sk IS NULL) +----------------------------PhysicalOlapScan[store_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ---------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------filter(ws_bill_addr_sk IS NULL) ---------------------------PhysicalOlapScan[web_sales] +--------------------------PhysicalOlapScan[date_dim] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +----------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +------------------------PhysicalProject +--------------------------filter(ws_bill_addr_sk IS NULL) +----------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) -----------------------PhysicalProject -------------------------filter(cs_warehouse_sk IS NULL) ---------------------------PhysicalOlapScan[catalog_sales] +--------------------------PhysicalOlapScan[date_dim] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +------------------------PhysicalProject +--------------------------filter(cs_warehouse_sk IS NULL) +----------------------------PhysicalOlapScan[catalog_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[item] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------PhysicalOlapScan[item] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out index 649b78703e9e23..6758459211b249 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query77.out @@ -1,107 +1,108 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_77 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashJoin[LEFT_OUTER_JOIN](ss.s_store_sk = sr.s_store_sk) -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashJoin[LEFT_OUTER_JOIN](ss.s_store_sk = sr.s_store_sk) +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_returns.sr_store_sk = store.s_store_sk) -----------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_returns] +----------------------------------------PhysicalOlapScan[store] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_returns.sr_store_sk = store.s_store_sk) +------------------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_returns] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date <= 1998-09-04)(date_dim.d_date >= 1998-08-05)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------NestedLoopJoin[CROSS_JOIN] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[catalog_sales] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) -----------------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute +----------------------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------NestedLoopJoin[CROSS_JOIN] ------------------------PhysicalProject --------------------------hashAgg[GLOBAL] ----------------------------PhysicalDistribute ------------------------------hashAgg[LOCAL] --------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------PhysicalOlapScan[catalog_sales] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject ----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) ------------------------------------------PhysicalOlapScan[date_dim] -------------------PhysicalProject ---------------------hashJoin[LEFT_OUTER_JOIN](ws.wp_web_page_sk = wr.wp_web_page_sk) -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +--------------------------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalProject +----------------------hashJoin[LEFT_OUTER_JOIN](ws.wp_web_page_sk = wr.wp_web_page_sk) +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_page] -----------------------PhysicalProject -------------------------hashAgg[GLOBAL] ---------------------------PhysicalDistribute -----------------------------hashAgg[LOCAL] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) -----------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_returns] +----------------------------------------PhysicalOlapScan[web_page] +------------------------PhysicalProject +--------------------------hashAgg[GLOBAL] +----------------------------PhysicalDistribute +------------------------------hashAgg[LOCAL] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_returns.wr_web_page_sk = web_page.wp_web_page_sk) +------------------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[web_returns] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) +--------------------------------------------PhysicalOlapScan[date_dim] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((date_dim.d_date >= 1998-08-05)(date_dim.d_date <= 1998-09-04)) -------------------------------------------PhysicalOlapScan[date_dim] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_page] +----------------------------------------PhysicalOlapScan[web_page] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out index 855043acbed68b..812bbb057959b8 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query78.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_78 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) -----------hashJoin[LEFT_OUTER_JOIN](cs.cs_sold_year = ss.ss_sold_year)(cs.cs_item_sk = ss.ss_item_sk)(cs.cs_customer_sk = ss.ss_customer_sk) -------------PhysicalProject ---------------hashJoin[LEFT_OUTER_JOIN](ws.ws_sold_year = ss.ss_sold_year)(ws.ws_item_sk = ss.ss_item_sk)(ws.ws_customer_sk = ss.ss_customer_sk) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------filter(sr_ticket_number IS NULL) ---------------------------------hashJoin[LEFT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_sales] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[store_returns] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------filter(wr_order_number IS NULL) ---------------------------------hashJoin[LEFT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_sales] -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[web_returns] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((date_dim.d_year = 2000)) -----------------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------filter(cr_order_number IS NULL) -----------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter(((coalesce(ws_qty, 0) > 0) OR (coalesce(cs_qty, 0) > 0))) +------------hashJoin[LEFT_OUTER_JOIN](cs.cs_sold_year = ss.ss_sold_year)(cs.cs_item_sk = ss.ss_item_sk)(cs.cs_customer_sk = ss.ss_customer_sk) +--------------PhysicalProject +----------------hashJoin[LEFT_OUTER_JOIN](ws.ws_sold_year = ss.ss_sold_year)(ws.ws_item_sk = ss.ss_item_sk)(ws.ws_customer_sk = ss.ss_customer_sk) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------filter(sr_ticket_number IS NULL) +----------------------------------hashJoin[LEFT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_sales] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[store_returns] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] -------------------------PhysicalDistribute +--------------------------------filter(wr_order_number IS NULL) +----------------------------------hashJoin[LEFT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_returns.wr_order_number = web_sales.ws_order_number) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_sales] +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[web_returns] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_year = 2000)) +------------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) --------------------------PhysicalProject -----------------------------filter((date_dim.d_year = 2000)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------filter(cr_order_number IS NULL) +------------------------------hashJoin[LEFT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_returns.cr_order_number = catalog_sales.cs_order_number) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_sales] +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((date_dim.d_year = 2000)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out index d51a2de9696c87..05f8bd4c577563 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query79.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_79 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](ms.ss_customer_sk = customer.c_customer_sk) -----------PhysicalDistribute -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](ms.ss_customer_sk = customer.c_customer_sk) +------------PhysicalDistribute +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter(d_year IN (1998, 1999, 2000)(date_dim.d_dow = 1)) +------------------------------------PhysicalOlapScan[date_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter(d_year IN (1998, 1999, 2000)(date_dim.d_dow = 1)) -----------------------------------PhysicalOlapScan[date_dim] +--------------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) +----------------------------------PhysicalOlapScan[household_demographics] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter(((household_demographics.hd_dep_count = 5) OR (household_demographics.hd_vehicle_count > 4))) ---------------------------------PhysicalOlapScan[household_demographics] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((store.s_number_employees >= 200)(store.s_number_employees <= 295)) -------------------------------PhysicalOlapScan[store] -----------PhysicalDistribute -------------PhysicalProject ---------------PhysicalOlapScan[customer] +------------------------------filter((store.s_number_employees >= 200)(store.s_number_employees <= 295)) +--------------------------------PhysicalOlapScan[store] +------------PhysicalDistribute +--------------PhysicalProject +----------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out index b00cd15207f6fc..2c6d8aff03ad51 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query8.out @@ -1,44 +1,45 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_8 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](expr_substring(s_zip, 1, 2) = expr_substring(ca_zip, 1, 2)) -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[store_sales] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((date_dim.d_year = 1998)(date_dim.d_qoy = 2)) -----------------------------PhysicalOlapScan[date_dim] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[store] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](expr_substring(s_zip, 1, 2) = expr_substring(ca_zip, 1, 2)) ------------------PhysicalProject ---------------------PhysicalIntersect -----------------------PhysicalDistribute +--------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------PhysicalProject ---------------------------filter(substring(ca_zip, 1, 5) IN ('47602', '16704', '35863', '28577', '83910', '36201', '58412', '48162', '28055', '41419', '80332', '38607', '77817', '24891', '16226', '18410', '21231', '59345', '13918', '51089', '20317', '17167', '54585', '67881', '78366', '47770', '18360', '51717', '73108', '14440', '21800', '89338', '45859', '65501', '34948', '25973', '73219', '25333', '17291', '10374', '18829', '60736', '82620', '41351', '52094', '19326', '25214', '54207', '40936', '21814', '79077', '25178', '75742', '77454', '30621', '89193', '27369', '41232', '48567', '83041', '71948', '37119', '68341', '14073', '16891', '62878', '49130', '19833', '24286', '27700', '40979', '50412', '81504', '94835', '84844', '71954', '39503', '57649', '18434', '24987', '12350', '86379', '27413', '44529', '98569', '16515', '27287', '24255', '21094', '16005', '56436', '91110', '68293', '56455', '54558', '10298', '83647', '32754', '27052', '51766', '19444', '13869', '45645', '94791', '57631', '20712', '37788', '41807', '46507', '21727', '71836', '81070', '50632', '88086', '63991', '20244', '31655', '51782', '29818', '63792', '68605', '94898', '36430', '57025', '20601', '82080', '33869', '22728', '35834', '29086', '92645', '98584', '98072', '11652', '78093', '57553', '43830', '71144', '53565', '18700', '90209', '71256', '38353', '54364', '28571', '96560', '57839', '56355', '50679', '45266', '84680', '34306', '34972', '48530', '30106', '15371', '92380', '84247', '92292', '68852', '13338', '34594', '82602', '70073', '98069', '85066', '47289', '11686', '98862', '26217', '47529', '63294', '51793', '35926', '24227', '14196', '24594', '32489', '99060', '49472', '43432', '49211', '14312', '88137', '47369', '56877', '20534', '81755', '15794', '12318', '21060', '73134', '41255', '63073', '81003', '73873', '66057', '51184', '51195', '45676', '92696', '70450', '90669', '98338', '25264', '38919', '59226', '58581', '60298', '17895', '19489', '52301', '80846', '95464', '68770', '51634', '19988', '18367', '18421', '11618', '67975', '25494', '41352', '95430', '15734', '62585', '97173', '33773', '10425', '75675', '53535', '17879', '41967', '12197', '67998', '79658', '59130', '72592', '14851', '43933', '68101', '50636', '25717', '71286', '24660', '58058', '72991', '95042', '15543', '33122', '69280', '11912', '59386', '27642', '65177', '17672', '33467', '64592', '36335', '54010', '18767', '63193', '42361', '49254', '33113', '33159', '36479', '59080', '11855', '81963', '31016', '49140', '29392', '41836', '32958', '53163', '13844', '73146', '23952', '65148', '93498', '14530', '46131', '58454', '13376', '13378', '83986', '12320', '17193', '59852', '46081', '98533', '52389', '13086', '68843', '31013', '13261', '60560', '13443', '45533', '83583', '11489', '58218', '19753', '22911', '25115', '86709', '27156', '32669', '13123', '51933', '39214', '41331', '66943', '14155', '69998', '49101', '70070', '35076', '14242', '73021', '59494', '15782', '29752', '37914', '74686', '83086', '34473', '15751', '81084', '49230', '91894', '60624', '17819', '28810', '63180', '56224', '39459', '55233', '75752', '43639', '55349', '86057', '62361', '50788', '31830', '58062', '18218', '85761', '60083', '45484', '21204', '90229', '70041', '41162', '35390', '16364', '39500', '68908', '26689', '52868', '81335', '40146', '11340', '61527', '61794', '71997', '30415', '59004', '29450', '58117', '69952', '33562', '83833', '27385', '61860', '96435', '48333', '23065', '32961', '84919', '61997', '99132', '22815', '56600', '68730', '48017', '95694', '32919', '88217', '27116', '28239', '58032', '18884', '16791', '21343', '97462', '18569', '75660', '15475')) -----------------------------PhysicalOlapScan[customer_address] +--------------------------PhysicalOlapScan[store_sales] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((date_dim.d_year = 1998)(date_dim.d_qoy = 2)) +------------------------------PhysicalOlapScan[date_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((cnt > 10)) -----------------------------hashAgg[GLOBAL] -------------------------------PhysicalDistribute ---------------------------------hashAgg[LOCAL] -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) ---------------------------------------PhysicalProject -----------------------------------------PhysicalOlapScan[customer_address] ---------------------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[store] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalIntersect +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter(substring(ca_zip, 1, 5) IN ('47602', '16704', '35863', '28577', '83910', '36201', '58412', '48162', '28055', '41419', '80332', '38607', '77817', '24891', '16226', '18410', '21231', '59345', '13918', '51089', '20317', '17167', '54585', '67881', '78366', '47770', '18360', '51717', '73108', '14440', '21800', '89338', '45859', '65501', '34948', '25973', '73219', '25333', '17291', '10374', '18829', '60736', '82620', '41351', '52094', '19326', '25214', '54207', '40936', '21814', '79077', '25178', '75742', '77454', '30621', '89193', '27369', '41232', '48567', '83041', '71948', '37119', '68341', '14073', '16891', '62878', '49130', '19833', '24286', '27700', '40979', '50412', '81504', '94835', '84844', '71954', '39503', '57649', '18434', '24987', '12350', '86379', '27413', '44529', '98569', '16515', '27287', '24255', '21094', '16005', '56436', '91110', '68293', '56455', '54558', '10298', '83647', '32754', '27052', '51766', '19444', '13869', '45645', '94791', '57631', '20712', '37788', '41807', '46507', '21727', '71836', '81070', '50632', '88086', '63991', '20244', '31655', '51782', '29818', '63792', '68605', '94898', '36430', '57025', '20601', '82080', '33869', '22728', '35834', '29086', '92645', '98584', '98072', '11652', '78093', '57553', '43830', '71144', '53565', '18700', '90209', '71256', '38353', '54364', '28571', '96560', '57839', '56355', '50679', '45266', '84680', '34306', '34972', '48530', '30106', '15371', '92380', '84247', '92292', '68852', '13338', '34594', '82602', '70073', '98069', '85066', '47289', '11686', '98862', '26217', '47529', '63294', '51793', '35926', '24227', '14196', '24594', '32489', '99060', '49472', '43432', '49211', '14312', '88137', '47369', '56877', '20534', '81755', '15794', '12318', '21060', '73134', '41255', '63073', '81003', '73873', '66057', '51184', '51195', '45676', '92696', '70450', '90669', '98338', '25264', '38919', '59226', '58581', '60298', '17895', '19489', '52301', '80846', '95464', '68770', '51634', '19988', '18367', '18421', '11618', '67975', '25494', '41352', '95430', '15734', '62585', '97173', '33773', '10425', '75675', '53535', '17879', '41967', '12197', '67998', '79658', '59130', '72592', '14851', '43933', '68101', '50636', '25717', '71286', '24660', '58058', '72991', '95042', '15543', '33122', '69280', '11912', '59386', '27642', '65177', '17672', '33467', '64592', '36335', '54010', '18767', '63193', '42361', '49254', '33113', '33159', '36479', '59080', '11855', '81963', '31016', '49140', '29392', '41836', '32958', '53163', '13844', '73146', '23952', '65148', '93498', '14530', '46131', '58454', '13376', '13378', '83986', '12320', '17193', '59852', '46081', '98533', '52389', '13086', '68843', '31013', '13261', '60560', '13443', '45533', '83583', '11489', '58218', '19753', '22911', '25115', '86709', '27156', '32669', '13123', '51933', '39214', '41331', '66943', '14155', '69998', '49101', '70070', '35076', '14242', '73021', '59494', '15782', '29752', '37914', '74686', '83086', '34473', '15751', '81084', '49230', '91894', '60624', '17819', '28810', '63180', '56224', '39459', '55233', '75752', '43639', '55349', '86057', '62361', '50788', '31830', '58062', '18218', '85761', '60083', '45484', '21204', '90229', '70041', '41162', '35390', '16364', '39500', '68908', '26689', '52868', '81335', '40146', '11340', '61527', '61794', '71997', '30415', '59004', '29450', '58117', '69952', '33562', '83833', '27385', '61860', '96435', '48333', '23065', '32961', '84919', '61997', '99132', '22815', '56600', '68730', '48017', '95694', '32919', '88217', '27116', '28239', '58032', '18884', '16791', '21343', '97462', '18569', '75660', '15475')) +------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((cnt > 10)) +------------------------------hashAgg[GLOBAL] +--------------------------------PhysicalDistribute +----------------------------------hashAgg[LOCAL] +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) ----------------------------------------PhysicalProject -------------------------------------------filter((cast(c_preferred_cust_flag as VARCHAR(*)) = 'Y')) ---------------------------------------------PhysicalOlapScan[customer] +------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((cast(c_preferred_cust_flag as VARCHAR(*)) = 'Y')) +----------------------------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out index 29730c3a44aa09..da407a03dc5a24 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query80.out @@ -1,102 +1,103 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_80 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalRepeat -----------------PhysicalUnion -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) -------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ---------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalRepeat +------------------PhysicalUnion +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[RIGHT_OUTER_JOIN](store_sales.ss_item_sk = store_returns.sr_item_sk)(store_sales.ss_ticket_number = store_returns.sr_ticket_number) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_promo_sk = promotion.p_promo_sk) +--------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[store_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) -----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------filter((item.i_current_price > 50.00)) +----------------------------------------------PhysicalOlapScan[item] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((item.i_current_price > 50.00)) ---------------------------------------------PhysicalOlapScan[item] +------------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) -------------------------------------------PhysicalOlapScan[promotion] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[catalog_returns] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_catalog_page_sk = catalog_page.cp_catalog_page_sk) -----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) -------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) ---------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[catalog_sales] +----------------------------------------PhysicalOlapScan[store] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[RIGHT_OUTER_JOIN](catalog_sales.cs_item_sk = catalog_returns.cr_item_sk)(catalog_sales.cs_order_number = catalog_returns.cr_order_number) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_catalog_page_sk = catalog_page.cp_catalog_page_sk) +------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_promo_sk = promotion.p_promo_sk) +--------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[catalog_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) -----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------filter((item.i_current_price > 50.00)) +----------------------------------------------PhysicalOlapScan[item] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((item.i_current_price > 50.00)) ---------------------------------------------PhysicalOlapScan[item] +------------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) -------------------------------------------PhysicalOlapScan[promotion] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[catalog_page] -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_returns] -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) -----------------------------------hashJoin[INNER_JOIN](web_sales.ws_promo_sk = promotion.p_promo_sk) -------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) ---------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------------------PhysicalProject -------------------------------------------PhysicalOlapScan[web_sales] +----------------------------------------PhysicalOlapScan[catalog_page] +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[RIGHT_OUTER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_returns] +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](web_sales.ws_web_site_sk = web_site.web_site_sk) +------------------------------------hashJoin[INNER_JOIN](web_sales.ws_promo_sk = promotion.p_promo_sk) +--------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = item.i_item_sk) +----------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------------PhysicalProject +--------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------PhysicalDistribute +--------------------------------------------PhysicalProject +----------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) +------------------------------------------------PhysicalOlapScan[date_dim] ----------------------------------------PhysicalDistribute ------------------------------------------PhysicalProject ---------------------------------------------filter((date_dim.d_date >= 1998-08-28)(date_dim.d_date <= 1998-09-27)) -----------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------filter((item.i_current_price > 50.00)) +----------------------------------------------PhysicalOlapScan[item] --------------------------------------PhysicalDistribute ----------------------------------------PhysicalProject -------------------------------------------filter((item.i_current_price > 50.00)) ---------------------------------------------PhysicalOlapScan[item] +------------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) +--------------------------------------------PhysicalOlapScan[promotion] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter((cast(p_channel_tv as VARCHAR(*)) = 'N')) -------------------------------------------PhysicalOlapScan[promotion] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[web_site] +----------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out index 11524e0f10bfcc..15caa8024db395 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query81.out @@ -19,28 +19,29 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------------PhysicalDistribute ----------------PhysicalProject ------------------PhysicalOlapScan[customer_address] ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) -------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) ---------------PhysicalDistribute -----------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------PhysicalDistribute -----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute +--PhysicalResultSink +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------hashJoin[LEFT_SEMI_JOIN](ctr1.ctr_state = ctr2.ctr_state)(cast(ctr_total_return as DOUBLE) > cast((avg(ctr_total_return) * 1.2) as DOUBLE)) +--------------hashJoin[INNER_JOIN](ctr1.ctr_customer_sk = customer.c_customer_sk) +----------------PhysicalDistribute +------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------PhysicalDistribute +------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) --------------------PhysicalProject -----------------------filter((cast(ca_state as VARCHAR(*)) = 'CA')) -------------------------PhysicalOlapScan[customer_address] -------------PhysicalDistribute ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((cast(ca_state as VARCHAR(*)) = 'CA')) +--------------------------PhysicalOlapScan[customer_address] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out index 6a27e9eece414e..c9976cb1eb7975 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query82.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_82 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[store_sales] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) -----------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) -------------------------PhysicalProject ---------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) -----------------------------PhysicalOlapScan[inventory] +--------------------PhysicalOlapScan[store_sales] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = inventory.inv_date_sk) +------------------------hashJoin[INNER_JOIN](inventory.inv_item_sk = item.i_item_sk) +--------------------------PhysicalProject +----------------------------filter((inventory.inv_quantity_on_hand >= 100)(inventory.inv_quantity_on_hand <= 500)) +------------------------------PhysicalOlapScan[inventory] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((item.i_current_price >= 17.00)i_manufact_id IN (138, 169, 339, 639)(item.i_current_price <= 47.00)) +--------------------------------PhysicalOlapScan[item] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((item.i_current_price >= 17.00)i_manufact_id IN (138, 169, 339, 639)(item.i_current_price <= 47.00)) -------------------------------PhysicalOlapScan[item] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((date_dim.d_date <= 1999-09-07)(date_dim.d_date >= 1999-07-09)) -----------------------------PhysicalOlapScan[date_dim] +----------------------------filter((date_dim.d_date <= 1999-09-07)(date_dim.d_date >= 1999-07-09)) +------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out index 67d4702363e19b..f1f3cfaf941eab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query83.out @@ -1,37 +1,11 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_83 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](sr_items.item_id = cr_items.item_id) -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) ---------------------------hashJoin[INNER_JOIN](catalog_returns.cr_item_sk = item.i_item_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_returns] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[item] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) -----------------------------------PhysicalOlapScan[date_dim] -----------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](sr_items.item_id = cr_items.item_id) ------------PhysicalProject --------------hashAgg[GLOBAL] ----------------PhysicalDistribute @@ -39,10 +13,10 @@ PhysicalTopN --------------------PhysicalProject ----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +----------------------------hashJoin[INNER_JOIN](catalog_returns.cr_item_sk = item.i_item_sk) ------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalOlapScan[catalog_returns] ------------------------------PhysicalDistribute --------------------------------PhysicalProject ----------------------------------PhysicalOlapScan[item] @@ -58,30 +32,57 @@ PhysicalTopN --------------------------------PhysicalProject ----------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) ------------------------------------PhysicalOlapScan[date_dim] -------------PhysicalProject ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) -----------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_returns] +------------hashJoin[INNER_JOIN](sr_items.item_id = wr_items.item_id) +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_returns.sr_returned_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_returns] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) +--------------------------------------PhysicalOlapScan[date_dim] +--------------PhysicalProject +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_date = date_dim.d_date) --------------------------PhysicalProject -----------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[date_dim] +----------------------------hashJoin[INNER_JOIN](web_returns.wr_returned_date_sk = date_dim.d_date_sk) +------------------------------hashJoin[INNER_JOIN](web_returns.wr_item_sk = item.i_item_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_returns] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[item] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) -------------------------------------PhysicalOlapScan[date_dim] +----------------------------------PhysicalOlapScan[date_dim] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[LEFT_SEMI_JOIN](date_dim.d_week_seq = date_dim.d_week_seq) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter(cast(d_date as DATETIMEV2(0)) IN (2001-06-06 00:00:00, 2001-09-02 00:00:00, 2001-11-11 00:00:00)) +--------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out index 79cda5d28acb37..710652c2f1b65e 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query84.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_84 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashJoin[INNER_JOIN](store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk) -----------PhysicalProject -------------PhysicalOlapScan[store_returns] -----------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashJoin[INNER_JOIN](store_returns.sr_cdemo_sk = customer_demographics.cd_demo_sk) ------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) -----------------PhysicalProject -------------------PhysicalOlapScan[customer_demographics] -----------------PhysicalDistribute +--------------PhysicalOlapScan[store_returns] +------------PhysicalDistribute +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[customer] ---------------------------PhysicalDistribute +--------------------PhysicalOlapScan[customer_demographics] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](customer.c_current_addr_sk = customer_address.ca_address_sk) ----------------------------PhysicalProject -------------------------------filter((customer_address.ca_city = 'Oakwood')) ---------------------------------PhysicalOlapScan[customer_address] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](income_band.ib_income_band_sk = household_demographics.hd_income_band_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[household_demographics] ---------------------------PhysicalDistribute +------------------------------PhysicalOlapScan[customer] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((customer_address.ca_city = 'Oakwood')) +----------------------------------PhysicalOlapScan[customer_address] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](income_band.ib_income_band_sk = household_demographics.hd_income_band_sk) ----------------------------PhysicalProject -------------------------------filter((cast(ib_upper_bound as BIGINT) <= 55806)(income_band.ib_lower_bound >= 5806)) ---------------------------------PhysicalOlapScan[income_band] +------------------------------PhysicalOlapScan[household_demographics] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((cast(ib_upper_bound as BIGINT) <= 55806)(income_band.ib_lower_bound >= 5806)) +----------------------------------PhysicalOlapScan[income_band] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out index e430099ada691c..77be6e85445279 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query85.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_85 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) -------------------PhysicalProject ---------------------PhysicalOlapScan[web_page] -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)(cd1.cd_marital_status = cd2.cd_marital_status)(cd1.cd_education_status = cd2.cd_education_status) --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](cd2.cd_demo_sk = web_returns.wr_returning_cdemo_sk)(cd1.cd_marital_status = cd2.cd_marital_status)(cd1.cd_education_status = cd2.cd_education_status) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalDistribute +----------------------PhysicalOlapScan[customer_demographics] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) --------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)(((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) -------------------------------PhysicalProject ---------------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'Secondary'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')))) -----------------------------------PhysicalOlapScan[customer_demographics] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](reason.r_reason_sk = web_returns.wr_reason_sk) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[reason] -----------------------------------PhysicalDistribute -------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)(((ca_state IN ('FL', 'TX', 'DE') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('IN', 'ND', 'ID') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('MT', 'IL', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))) ---------------------------------------PhysicalProject -----------------------------------------filter(((ca_state IN ('FL', 'TX', 'DE') OR ca_state IN ('IN', 'ND', 'ID')) OR ca_state IN ('MT', 'IL', 'OH'))(customer_address.ca_country = 'United States')) -------------------------------------------PhysicalOlapScan[customer_address] ---------------------------------------PhysicalDistribute +----------------------------PhysicalOlapScan[web_page] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](cd1.cd_demo_sk = web_returns.wr_refunded_cdemo_sk)(((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) AND ((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'Secondary')) AND ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00)))) OR (((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')) AND ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) +--------------------------------PhysicalProject +----------------------------------filter(((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = '4 yr Degree')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'S') AND (cast(cd_education_status as VARCHAR(*)) = 'Secondary'))) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')))) +------------------------------------PhysicalOlapScan[customer_demographics] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](reason.r_reason_sk = web_returns.wr_reason_sk) +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[reason] +------------------------------------PhysicalDistribute +--------------------------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = web_returns.wr_refunded_addr_sk)(((ca_state IN ('FL', 'TX', 'DE') AND ((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00))) OR (ca_state IN ('IN', 'ND', 'ID') AND ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00)))) OR (ca_state IN ('MT', 'IL', 'OH') AND ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))) ----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[web_returns] ---------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) +------------------------------------------filter(((ca_state IN ('FL', 'TX', 'DE') OR ca_state IN ('IN', 'ND', 'ID')) OR ca_state IN ('MT', 'IL', 'OH'))(customer_address.ca_country = 'United States')) +--------------------------------------------PhysicalOlapScan[customer_address] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_item_sk = web_returns.wr_item_sk)(web_sales.ws_order_number = web_returns.wr_order_number) ----------------------------------------------PhysicalProject -------------------------------------------------filter(((((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00)) OR ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00))) OR ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))((((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00)) OR ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00))) OR ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) ---------------------------------------------------PhysicalOlapScan[web_sales] -----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalOlapScan[web_returns] +----------------------------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------------------------PhysicalProject ---------------------------------------------------filter((date_dim.d_year = 2000)) -----------------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------------------------filter(((((web_sales.ws_net_profit >= 100.00) AND (web_sales.ws_net_profit <= 200.00)) OR ((web_sales.ws_net_profit >= 150.00) AND (web_sales.ws_net_profit <= 300.00))) OR ((web_sales.ws_net_profit >= 50.00) AND (web_sales.ws_net_profit <= 250.00)))((((web_sales.ws_sales_price >= 100.00) AND (web_sales.ws_sales_price <= 150.00)) OR ((web_sales.ws_sales_price >= 50.00) AND (web_sales.ws_sales_price <= 100.00))) OR ((web_sales.ws_sales_price >= 150.00) AND (web_sales.ws_sales_price <= 200.00)))) +----------------------------------------------------PhysicalOlapScan[web_sales] +------------------------------------------------PhysicalDistribute +--------------------------------------------------PhysicalProject +----------------------------------------------------filter((date_dim.d_year = 2000)) +------------------------------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out index 6bb40cfe302c87..63e6ce5ba71dab 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query86.out @@ -1,28 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_86 -- -PhysicalProject ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalRepeat ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) -------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = web_sales.ws_sold_date_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] +PhysicalResultSink +--PhysicalProject +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalRepeat +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) +--------------------------------hashJoin[INNER_JOIN](d1.d_date_sk = web_sales.ws_sold_date_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((d1.d_month_seq <= 1235)(d1.d_month_seq >= 1224)) +----------------------------------------PhysicalOlapScan[date_dim] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((d1.d_month_seq <= 1235)(d1.d_month_seq >= 1224)) ---------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[item] +------------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out index 4b5af9e2a8ce95..c42693e374ca2f 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query87.out @@ -1,62 +1,63 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_87 -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------PhysicalExcept -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[store_sales] -----------------------------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------PhysicalExcept +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](store_sales.ss_customer_sk = customer.c_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[catalog_sales] -----------------------------PhysicalDistribute +--------------------------------PhysicalOlapScan[store_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_bill_customer_sk = customer.c_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] -----------PhysicalProject -------------hashAgg[GLOBAL] ---------------PhysicalDistribute -----------------hashAgg[LOCAL] -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_sales] -----------------------------PhysicalDistribute +--------------------------------PhysicalOlapScan[catalog_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_month_seq >= 1184)(date_dim.d_month_seq <= 1195)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] +------------PhysicalProject +--------------hashAgg[GLOBAL] +----------------PhysicalDistribute +------------------hashAgg[LOCAL] +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](web_sales.ws_bill_customer_sk = customer.c_customer_sk) +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](web_sales.ws_sold_date_sk = date_dim.d_date_sk) ------------------------------PhysicalProject ---------------------------------filter((date_dim.d_month_seq <= 1195)(date_dim.d_month_seq >= 1184)) -----------------------------------PhysicalOlapScan[date_dim] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[customer] +--------------------------------PhysicalOlapScan[web_sales] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((date_dim.d_month_seq <= 1195)(date_dim.d_month_seq >= 1184)) +------------------------------------PhysicalOlapScan[date_dim] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out index 5242db935c1e2e..e8b4b57dc82f4b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query88.out @@ -1,34 +1,57 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_88 -- -PhysicalProject ---NestedLoopJoin[CROSS_JOIN] +PhysicalResultSink +--PhysicalProject ----NestedLoopJoin[CROSS_JOIN] ------NestedLoopJoin[CROSS_JOIN] --------NestedLoopJoin[CROSS_JOIN] ----------NestedLoopJoin[CROSS_JOIN] ------------NestedLoopJoin[CROSS_JOIN] --------------NestedLoopJoin[CROSS_JOIN] -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) ---------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -----------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[store_sales] +----------------NestedLoopJoin[CROSS_JOIN] +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[store_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) +--------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) -------------------------------------PhysicalOlapScan[time_dim] +----------------------------------filter((store.s_store_name = 'ese')) +------------------------------------PhysicalOlapScan[store] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((store.s_store_name = 'ese')) -----------------------------------PhysicalOlapScan[store] ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) ---------------------------------PhysicalOlapScan[household_demographics] +--------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) +----------------------------------PhysicalOlapScan[household_demographics] +------------------PhysicalDistribute +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((time_dim.t_hour = 9)(time_dim.t_minute < 30)) +----------------------------------------PhysicalOlapScan[time_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((store.s_store_name = 'ese')) +--------------------------------------PhysicalOlapScan[store] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) +------------------------------------PhysicalOlapScan[household_demographics] ----------------PhysicalDistribute ------------------hashAgg[GLOBAL] --------------------PhysicalDistribute @@ -41,7 +64,7 @@ PhysicalProject ----------------------------------PhysicalOlapScan[store_sales] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((time_dim.t_hour = 9)(time_dim.t_minute < 30)) +------------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 9)) --------------------------------------PhysicalOlapScan[time_dim] ------------------------------PhysicalDistribute --------------------------------PhysicalProject @@ -63,7 +86,7 @@ PhysicalProject --------------------------------PhysicalOlapScan[store_sales] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 9)) +----------------------------------filter((time_dim.t_hour = 10)(time_dim.t_minute < 30)) ------------------------------------PhysicalOlapScan[time_dim] ----------------------------PhysicalDistribute ------------------------------PhysicalProject @@ -85,7 +108,7 @@ PhysicalProject ------------------------------PhysicalOlapScan[store_sales] ----------------------------PhysicalDistribute ------------------------------PhysicalProject ---------------------------------filter((time_dim.t_hour = 10)(time_dim.t_minute < 30)) +--------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 10)) ----------------------------------PhysicalOlapScan[time_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject @@ -107,7 +130,7 @@ PhysicalProject ----------------------------PhysicalOlapScan[store_sales] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 10)) +------------------------------filter((time_dim.t_minute < 30)(time_dim.t_hour = 11)) --------------------------------PhysicalOlapScan[time_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject @@ -129,7 +152,7 @@ PhysicalProject --------------------------PhysicalOlapScan[store_sales] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((time_dim.t_minute < 30)(time_dim.t_hour = 11)) +----------------------------filter((time_dim.t_hour = 11)(time_dim.t_minute >= 30)) ------------------------------PhysicalOlapScan[time_dim] ----------------------PhysicalDistribute ------------------------PhysicalProject @@ -151,7 +174,7 @@ PhysicalProject ------------------------PhysicalOlapScan[store_sales] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((time_dim.t_hour = 11)(time_dim.t_minute >= 30)) +--------------------------filter((time_dim.t_hour = 12)(time_dim.t_minute < 30)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject @@ -161,26 +184,4 @@ PhysicalProject --------------------PhysicalProject ----------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) ------------------------PhysicalOlapScan[household_demographics] -----PhysicalDistribute -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -----------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((time_dim.t_hour = 12)(time_dim.t_minute < 30)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((store.s_store_name = 'ese')) -------------------------PhysicalOlapScan[store] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter(((((household_demographics.hd_dep_count = -1) AND (household_demographics.hd_vehicle_count <= 1)) OR ((household_demographics.hd_dep_count = 4) AND (household_demographics.hd_vehicle_count <= 6))) OR ((household_demographics.hd_dep_count = 3) AND (household_demographics.hd_vehicle_count <= 5)))) -----------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out index 52df2ddd7bdb49..fff718f5872332 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query89.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_89 -- -PhysicalProject ---PhysicalTopN -----PhysicalDistribute -------PhysicalTopN ---------PhysicalProject -----------filter((CASE WHEN ( not (avg_monthly_sales = 0.0000)) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)) -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashAgg[GLOBAL] -----------------------PhysicalDistribute -------------------------hashAgg[LOCAL] ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) -------------------------------PhysicalProject ---------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) -----------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalProject +----PhysicalTopN +------PhysicalDistribute +--------PhysicalTopN +----------PhysicalProject +------------filter((CASE WHEN ( not (avg_monthly_sales = 0.0000)) THEN (abs((cast(sum_sales as DOUBLE) - cast(avg_monthly_sales as DOUBLE))) / cast(avg_monthly_sales as DOUBLE)) ELSE NULL END > 0.1)) +--------------PhysicalWindow +----------------PhysicalQuickSort +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +--------------------------------PhysicalProject +----------------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +------------------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalDistribute +----------------------------------------PhysicalProject +------------------------------------------filter(((i_category IN ('Jewelry', 'Shoes', 'Electronics') AND i_class IN ('semi-precious', 'athletic', 'portable')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'rock', 'maternity')))) +--------------------------------------------PhysicalOlapScan[item] ------------------------------------PhysicalDistribute --------------------------------------PhysicalProject -----------------------------------------filter(((i_category IN ('Jewelry', 'Shoes', 'Electronics') AND i_class IN ('semi-precious', 'athletic', 'portable')) OR (i_category IN ('Men', 'Music', 'Women') AND i_class IN ('accessories', 'rock', 'maternity')))) -------------------------------------------PhysicalOlapScan[item] -----------------------------------PhysicalDistribute -------------------------------------PhysicalProject ---------------------------------------filter((date_dim.d_year = 1999)) -----------------------------------------PhysicalOlapScan[date_dim] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[store] +----------------------------------------filter((date_dim.d_year = 1999)) +------------------------------------------PhysicalOlapScan[date_dim] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out index ba0301464369fb..b245af7262a6eb 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query9.out @@ -1,6 +1,6 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_9 -- -PhysicalDistribute +PhysicalResultSink --PhysicalProject ----NestedLoopJoin[CROSS_JOIN] ------NestedLoopJoin[CROSS_JOIN] @@ -18,17 +18,17 @@ PhysicalDistribute ------------------------------NestedLoopJoin[CROSS_JOIN] --------------------------------PhysicalProject ----------------------------------NestedLoopJoin[CROSS_JOIN] -------------------------------------PhysicalProject ---------------------------------------filter((reason.r_reason_sk = 1)) -----------------------------------------PhysicalOlapScan[reason] +------------------------------------PhysicalAssertNumRows +--------------------------------------hashAgg[GLOBAL] +----------------------------------------PhysicalDistribute +------------------------------------------hashAgg[LOCAL] +--------------------------------------------PhysicalProject +----------------------------------------------filter((store_sales.ss_quantity <= 20)(store_sales.ss_quantity >= 1)) +------------------------------------------------PhysicalOlapScan[store_sales] ------------------------------------PhysicalDistribute ---------------------------------------PhysicalAssertNumRows -----------------------------------------hashAgg[GLOBAL] -------------------------------------------PhysicalDistribute ---------------------------------------------hashAgg[LOCAL] -----------------------------------------------PhysicalProject -------------------------------------------------filter((store_sales.ss_quantity <= 20)(store_sales.ss_quantity >= 1)) ---------------------------------------------------PhysicalOlapScan[store_sales] +--------------------------------------PhysicalProject +----------------------------------------filter((reason.r_reason_sk = 1)) +------------------------------------------PhysicalOlapScan[reason] --------------------------------PhysicalDistribute ----------------------------------PhysicalAssertNumRows ------------------------------------hashAgg[GLOBAL] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out index 309483f6448848..a2d4fe07ad7a1c 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query90.out @@ -1,31 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_90 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----PhysicalProject -------NestedLoopJoin[CROSS_JOIN] ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) -------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) ---------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) -----------------------PhysicalProject -------------------------PhysicalOlapScan[web_sales] -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) -----------------------------PhysicalOlapScan[web_page] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((time_dim.t_hour >= 10)(time_dim.t_hour <= 11)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((household_demographics.hd_dep_count = 2)) -------------------------PhysicalOlapScan[household_demographics] ---------PhysicalDistribute +----PhysicalTopN +------PhysicalProject +--------NestedLoopJoin[CROSS_JOIN] ----------hashAgg[GLOBAL] ------------PhysicalDistribute --------------hashAgg[LOCAL] @@ -41,10 +20,32 @@ PhysicalTopN ------------------------------PhysicalOlapScan[web_page] ----------------------PhysicalDistribute ------------------------PhysicalProject ---------------------------filter((time_dim.t_hour >= 16)(time_dim.t_hour <= 17)) +--------------------------filter((time_dim.t_hour >= 10)(time_dim.t_hour <= 11)) ----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject ------------------------filter((household_demographics.hd_dep_count = 2)) --------------------------PhysicalOlapScan[household_demographics] +----------PhysicalDistribute +------------hashAgg[GLOBAL] +--------------PhysicalDistribute +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](web_sales.ws_ship_hdemo_sk = household_demographics.hd_demo_sk) +----------------------hashJoin[INNER_JOIN](web_sales.ws_sold_time_sk = time_dim.t_time_sk) +------------------------hashJoin[INNER_JOIN](web_sales.ws_web_page_sk = web_page.wp_web_page_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[web_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((web_page.wp_char_count >= 5000)(web_page.wp_char_count <= 5200)) +--------------------------------PhysicalOlapScan[web_page] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((time_dim.t_hour >= 16)(time_dim.t_hour <= 17)) +------------------------------PhysicalOlapScan[time_dim] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((household_demographics.hd_dep_count = 2)) +----------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out index f856ec074871f7..1e3674e05365b9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query91.out @@ -1,47 +1,48 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_91 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) -------------------PhysicalProject ---------------------filter((customer_address.ca_gmt_offset = -6.00)) -----------------------PhysicalOlapScan[customer_address] -------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](customer_address.ca_address_sk = customer.c_current_addr_sk) --------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')))) -------------------------------PhysicalOlapScan[customer_demographics] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) +----------------------filter((customer_address.ca_gmt_offset = -6.00)) +------------------------PhysicalOlapScan[customer_address] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](customer_demographics.cd_demo_sk = customer.c_current_cdemo_sk) +--------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returning_customer_sk = customer.c_customer_sk) ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[customer] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) ---------------------------------------PhysicalProject -----------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk) -------------------------------------------PhysicalProject ---------------------------------------------PhysicalOlapScan[catalog_returns] -------------------------------------------PhysicalDistribute ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[call_center] ---------------------------------------PhysicalDistribute -----------------------------------------PhysicalProject -------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2001)) ---------------------------------------------PhysicalOlapScan[date_dim] -----------------------------PhysicalDistribute +------------------------------filter((((cast(cd_marital_status as VARCHAR(*)) = 'M') AND (cast(cd_education_status as VARCHAR(*)) = 'Unknown')) OR ((cast(cd_marital_status as VARCHAR(*)) = 'W') AND (cast(cd_education_status as VARCHAR(*)) = 'Advanced Degree')))) +--------------------------------PhysicalOlapScan[customer_demographics] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](household_demographics.hd_demo_sk = customer.c_current_hdemo_sk) ------------------------------PhysicalProject ---------------------------------filter((hd_buy_potential like '1001-5000%')) -----------------------------------PhysicalOlapScan[household_demographics] +--------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returning_customer_sk = customer.c_customer_sk) +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[customer] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_returned_date_sk = date_dim.d_date_sk) +----------------------------------------PhysicalProject +------------------------------------------hashJoin[INNER_JOIN](catalog_returns.cr_call_center_sk = call_center.cc_call_center_sk) +--------------------------------------------PhysicalProject +----------------------------------------------PhysicalOlapScan[catalog_returns] +--------------------------------------------PhysicalDistribute +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[call_center] +----------------------------------------PhysicalDistribute +------------------------------------------PhysicalProject +--------------------------------------------filter((date_dim.d_moy = 11)(date_dim.d_year = 2001)) +----------------------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((hd_buy_potential like '1001-5000%')) +------------------------------------PhysicalOlapScan[household_demographics] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out index a37b9b6b369669..af107f55b49ec4 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query92.out @@ -1,26 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_92 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------filter((cast(ws_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(ws_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) ---------------PhysicalWindow -----------------PhysicalQuickSort -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) -------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_sales] +----PhysicalTopN +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------filter((cast(ws_ext_discount_amt as DECIMALV3(38, 5)) > (1.3 * avg(cast(ws_ext_discount_amt as DECIMALV3(9, 4))) OVER(PARTITION BY i_item_sk)))) +----------------PhysicalWindow +------------------PhysicalQuickSort +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](date_dim.d_date_sk = web_sales.ws_sold_date_sk) +--------------------------hashJoin[INNER_JOIN](item.i_item_sk = web_sales.ws_item_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((item.i_manufact_id = 320)) +----------------------------------PhysicalOlapScan[item] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((item.i_manufact_id = 320)) ---------------------------------PhysicalOlapScan[item] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter((date_dim.d_date <= 2002-05-27)(date_dim.d_date >= 2002-02-26)) -------------------------------PhysicalOlapScan[date_dim] +------------------------------filter((date_dim.d_date <= 2002-05-27)(date_dim.d_date >= 2002-02-26)) +--------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out index 4dca61b5d01296..0ead46c8d739e0 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query93.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_93 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = store_sales.ss_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) -----------------PhysicalProject -------------------PhysicalOlapScan[store_sales] -----------------hashJoin[INNER_JOIN](store_returns.sr_reason_sk = reason.r_reason_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](store_returns.sr_item_sk = store_sales.ss_item_sk)(store_returns.sr_ticket_number = store_sales.ss_ticket_number) ------------------PhysicalProject ---------------------PhysicalOlapScan[store_returns] -------------------PhysicalDistribute +--------------------PhysicalOlapScan[store_sales] +------------------hashJoin[INNER_JOIN](store_returns.sr_reason_sk = reason.r_reason_sk) --------------------PhysicalProject -----------------------filter((cast(r_reason_desc as VARCHAR(*)) = 'duplicate purchase')) -------------------------PhysicalOlapScan[reason] +----------------------PhysicalOlapScan[store_returns] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------filter((cast(r_reason_desc as VARCHAR(*)) = 'duplicate purchase')) +--------------------------PhysicalOlapScan[reason] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out index c26035693e5596..b613247de922e9 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query94.out @@ -1,37 +1,38 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_94 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----PhysicalProject -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) -----------------PhysicalProject -------------------filter((date_dim.d_date >= 2000-02-01)(cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('2000-2-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))) ---------------------PhysicalOlapScan[date_dim] -----------------PhysicalDistribute +----PhysicalTopN +------PhysicalProject +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) ------------------PhysicalProject ---------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) -----------------------PhysicalDistribute -------------------------PhysicalProject ---------------------------PhysicalOlapScan[web_sales] -----------------------hashJoin[RIGHT_ANTI_JOIN](ws1.ws_order_number = wr1.wr_order_number) +--------------------filter((date_dim.d_date >= 2000-02-01)(cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('2000-2-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))) +----------------------PhysicalOlapScan[date_dim] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws2.ws_order_number)( not (ws_warehouse_sk = ws_warehouse_sk)) ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------PhysicalOlapScan[web_returns] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) -----------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[web_sales] +----------------------------PhysicalOlapScan[web_sales] +------------------------hashJoin[RIGHT_ANTI_JOIN](ws1.ws_order_number = wr1.wr_order_number) +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[web_returns] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) +------------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[web_sales] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'OK')) +--------------------------------------PhysicalOlapScan[customer_address] ------------------------------PhysicalDistribute --------------------------------PhysicalProject -----------------------------------filter((cast(ca_state as VARCHAR(*)) = 'OK')) -------------------------------------PhysicalOlapScan[customer_address] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((cast(web_company_name as VARCHAR(*)) = 'pri')) -----------------------------------PhysicalOlapScan[web_site] +----------------------------------filter((cast(web_company_name as VARCHAR(*)) = 'pri')) +------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out index 410beed37167a1..f13ff715af4682 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query95.out @@ -10,43 +10,44 @@ PhysicalCteAnchor ( cteId=CTEId#0 ) --------PhysicalDistribute ----------PhysicalProject ------------PhysicalOlapScan[web_sales] ---PhysicalTopN +--PhysicalResultSink ----PhysicalTopN -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) -------------------PhysicalProject ---------------------filter((date_dim.d_date >= 1999-02-01)(cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('1999-2-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))) -----------------------PhysicalOlapScan[date_dim] -------------------PhysicalDistribute ---------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = web_returns.wr_order_number) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](web_returns.wr_order_number = ws_wh.ws_order_number) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[web_returns] -----------------------PhysicalDistribute -------------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws_wh.ws_order_number) ---------------------------PhysicalDistribute -----------------------------PhysicalProject -------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) -------------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[web_sales] +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](ws1.ws_ship_date_sk = date_dim.d_date_sk) +--------------------PhysicalProject +----------------------filter((date_dim.d_date >= 1999-02-01)(cast(d_date as DATETIMEV2(0)) <= cast(days_add(cast('1999-2-01' as DATEV2), INTERVAL 60 DAY) as DATETIMEV2(0)))) +------------------------PhysicalOlapScan[date_dim] +--------------------PhysicalDistribute +----------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = web_returns.wr_order_number) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](web_returns.wr_order_number = ws_wh.ws_order_number) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[web_returns] +------------------------PhysicalDistribute +--------------------------hashJoin[RIGHT_SEMI_JOIN](ws1.ws_order_number = ws_wh.ws_order_number) +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------PhysicalCteConsumer ( cteId=CTEId#0 ) +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](ws1.ws_web_site_sk = web_site.web_site_sk) +--------------------------------hashJoin[INNER_JOIN](ws1.ws_ship_addr_sk = customer_address.ca_address_sk) +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[web_sales] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'NC')) +----------------------------------------PhysicalOlapScan[customer_address] --------------------------------PhysicalDistribute ----------------------------------PhysicalProject -------------------------------------filter((cast(ca_state as VARCHAR(*)) = 'NC')) ---------------------------------------PhysicalOlapScan[customer_address] -------------------------------PhysicalDistribute ---------------------------------PhysicalProject -----------------------------------filter((cast(web_company_name as VARCHAR(*)) = 'pri')) -------------------------------------PhysicalOlapScan[web_site] +------------------------------------filter((cast(web_company_name as VARCHAR(*)) = 'pri')) +--------------------------------------PhysicalOlapScan[web_site] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out index d935628ed6dabb..6d7df70b717438 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query96.out @@ -1,27 +1,28 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_96 -- -PhysicalTopN +PhysicalResultSink --PhysicalTopN -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) -------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) ---------------------PhysicalProject -----------------------PhysicalOlapScan[store_sales] +----PhysicalTopN +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN](store_sales.ss_store_sk = store.s_store_sk) +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](store_sales.ss_hdemo_sk = household_demographics.hd_demo_sk) +--------------------hashJoin[INNER_JOIN](store_sales.ss_sold_time_sk = time_dim.t_time_sk) +----------------------PhysicalProject +------------------------PhysicalOlapScan[store_sales] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) +----------------------------PhysicalOlapScan[time_dim] --------------------PhysicalDistribute ----------------------PhysicalProject -------------------------filter((time_dim.t_minute >= 30)(time_dim.t_hour = 8)) ---------------------------PhysicalOlapScan[time_dim] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------filter((household_demographics.hd_dep_count = 3)) -------------------------PhysicalOlapScan[household_demographics] ---------------PhysicalDistribute -----------------PhysicalProject -------------------filter((store.s_store_name = 'ese')) ---------------------PhysicalOlapScan[store] +------------------------filter((household_demographics.hd_dep_count = 3)) +--------------------------PhysicalOlapScan[household_demographics] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((store.s_store_name = 'ese')) +----------------------PhysicalOlapScan[store] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out index fee94a11417d07..8b918462450d96 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query97.out @@ -1,34 +1,35 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_97 -- -PhysicalLimit +PhysicalResultSink --PhysicalLimit -----hashAgg[GLOBAL] -------PhysicalDistribute ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[FULL_OUTER_JOIN](ssci.customer_sk = csci.customer_sk)(ssci.item_sk = csci.item_sk) ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] ---------------------------PhysicalDistribute +----PhysicalLimit +------hashAgg[GLOBAL] +--------PhysicalDistribute +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[FULL_OUTER_JOIN](ssci.customer_sk = csci.customer_sk)(ssci.item_sk = csci.item_sk) +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) ---------------------------------PhysicalOlapScan[date_dim] ---------------PhysicalProject -----------------hashAgg[GLOBAL] -------------------PhysicalDistribute ---------------------hashAgg[LOCAL] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[catalog_sales] ---------------------------PhysicalDistribute +------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) +----------------------------------PhysicalOlapScan[date_dim] +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](catalog_sales.cs_sold_date_sk = date_dim.d_date_sk) ----------------------------PhysicalProject -------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) ---------------------------------PhysicalOlapScan[date_dim] +------------------------------PhysicalOlapScan[catalog_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_month_seq >= 1214)(date_dim.d_month_seq <= 1225)) +----------------------------------PhysicalOlapScan[date_dim] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out index 0e7c482e070ade..3a1c89d6a5fa6a 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query98.out @@ -1,26 +1,27 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_98 -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------PhysicalWindow -----------PhysicalQuickSort -------------PhysicalDistribute ---------------hashAgg[GLOBAL] -----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) -------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[store_sales] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------PhysicalWindow +------------PhysicalQuickSort +--------------PhysicalDistribute +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------hashJoin[INNER_JOIN](store_sales.ss_item_sk = item.i_item_sk) +--------------------------hashJoin[INNER_JOIN](store_sales.ss_sold_date_sk = date_dim.d_date_sk) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[store_sales] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((date_dim.d_date <= 2002-06-19)(date_dim.d_date >= 2002-05-20)) +----------------------------------PhysicalOlapScan[date_dim] --------------------------PhysicalDistribute ----------------------------PhysicalProject -------------------------------filter((date_dim.d_date <= 2002-06-19)(date_dim.d_date >= 2002-05-20)) ---------------------------------PhysicalOlapScan[date_dim] -------------------------PhysicalDistribute ---------------------------PhysicalProject -----------------------------filter(i_category IN ('Sports', 'Music', 'Shoes')) -------------------------------PhysicalOlapScan[item] +------------------------------filter(i_category IN ('Sports', 'Music', 'Shoes')) +--------------------------------PhysicalOlapScan[item] diff --git a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out index e48e37418c9d25..4f20f7bf82aa1b 100644 --- a/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out +++ b/regression-test/data/nereids_tpcds_shape_sf100_p0/shape/query99.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !ds_shape_99 -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) -----------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](catalog_sales.cs_warehouse_sk = warehouse.w_warehouse_sk) ------------------PhysicalProject --------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_mode_sk = ship_mode.sm_ship_mode_sk) -----------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = date_dim.d_date_sk) -------------------------PhysicalProject ---------------------------PhysicalOlapScan[catalog_sales] +----------------------hashJoin[INNER_JOIN](catalog_sales.cs_call_center_sk = call_center.cc_call_center_sk) +------------------------hashJoin[INNER_JOIN](catalog_sales.cs_ship_date_sk = date_dim.d_date_sk) +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[catalog_sales] +--------------------------PhysicalDistribute +----------------------------PhysicalProject +------------------------------filter((date_dim.d_month_seq >= 1224)(date_dim.d_month_seq <= 1235)) +--------------------------------PhysicalOlapScan[date_dim] ------------------------PhysicalDistribute --------------------------PhysicalProject -----------------------------filter((date_dim.d_month_seq >= 1224)(date_dim.d_month_seq <= 1235)) -------------------------------PhysicalOlapScan[date_dim] +----------------------------PhysicalOlapScan[call_center] ----------------------PhysicalDistribute ------------------------PhysicalProject --------------------------PhysicalOlapScan[ship_mode] ------------------PhysicalDistribute --------------------PhysicalProject -----------------------PhysicalOlapScan[call_center] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[warehouse] +----------------------PhysicalOlapScan[warehouse] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q1.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q1.out index 00a7cf50f83f52..7740edd76cc3a5 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q1.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q1.out @@ -1,12 +1,13 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((lineitem.l_shipdate <= 1998-09-02)) -----------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------filter((lineitem.l_shipdate <= 1998-09-02)) +------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out index c1d1c3548aedb4..bbf108f0aedc48 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q10.out @@ -1,25 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] ----------------PhysicalProject -------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------PhysicalProject -------------------------filter((lineitem.l_returnflag = 'R')) ---------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) ---------------------------PhysicalOlapScan[orders] ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalOlapScan[nation] +------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +--------------------PhysicalProject +----------------------filter((lineitem.l_returnflag = 'R')) +------------------------PhysicalOlapScan[lineitem] +--------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) +----------------------------------PhysicalOlapScan[orders] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out index cf27cdd93f4d44..914f6976ee3744 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q11.out @@ -1,39 +1,40 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * ps_availqty)) * 0.000002) as DOUBLE)) -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------PhysicalProject -------------------PhysicalOlapScan[partsupp] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[supplier] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((nation.n_name = 'GERMANY')) ---------------------------PhysicalOlapScan[nation] -----------PhysicalDistribute -------------PhysicalAssertNumRows +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * ps_availqty)) * 0.000002) as DOUBLE)) +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[GLOBAL] +----------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------PhysicalProject +--------------------PhysicalOlapScan[partsupp] ------------------PhysicalDistribute ---------------------hashAgg[LOCAL] +--------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute +------------------------PhysicalOlapScan[supplier] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((nation.n_name = 'GERMANY')) +----------------------------PhysicalOlapScan[nation] +------------PhysicalDistribute +--------------PhysicalAssertNumRows +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------------------------PhysicalProject -----------------------------------filter((nation.n_name = 'GERMANY')) -------------------------------------PhysicalOlapScan[nation] +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((nation.n_name = 'GERMANY')) +--------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out index cf83a4373e7a7d..1fec7c6597fd30 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q12.out @@ -1,16 +1,17 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------PhysicalProject -------------------PhysicalOlapScan[orders] -----------------PhysicalProject -------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) ---------------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------PhysicalProject +--------------------PhysicalOlapScan[orders] +------------------PhysicalProject +--------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out index ae50570ecf54f2..ab3e4ce172b9db 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q13.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(( not (o_comment like '%special%requests%'))) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(( not (o_comment like '%special%requests%'))) +----------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out index 9ec9d4f3fa9854..747b102ab8889a 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q14.out @@ -1,15 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalProject ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) -------------PhysicalProject ---------------PhysicalOlapScan[part] -------------PhysicalDistribute +PhysicalResultSink +--PhysicalProject +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) --------------PhysicalProject -----------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) -------------------PhysicalOlapScan[lineitem] +----------------PhysicalOlapScan[part] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) +--------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out index da9dd396bebe3d..4106594748a355 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q15.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) -----------PhysicalProject -------------PhysicalOlapScan[supplier] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) ---------------hashAgg[GLOBAL] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) +------------PhysicalProject +--------------PhysicalOlapScan[supplier] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +--------------------------PhysicalOlapScan[lineitem] ----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute -----------------PhysicalAssertNumRows -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalAssertNumRows +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +--------------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out index d72afcf57daec7..21332b6f99dc89 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q16.out @@ -1,21 +1,22 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] -------------------PhysicalProject ---------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) -----------------------PhysicalOlapScan[part] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((s_comment like '%Customer%Complaints%')) -----------------------PhysicalOlapScan[supplier] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +--------------------PhysicalProject +----------------------PhysicalOlapScan[partsupp] +--------------------PhysicalProject +----------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) +------------------------PhysicalOlapScan[part] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((s_comment like '%Customer%Complaints%')) +------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out index 90b6d8a1903660..65bad6d59809a4 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q17.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalProject ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalProject +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) +--------------PhysicalWindow +----------------PhysicalQuickSort +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ------------------------PhysicalProject ---------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) -----------------------------PhysicalOlapScan[part] +--------------------------PhysicalOlapScan[lineitem] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) +------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out index e91c302aaa36cb..ae401ef1cd429f 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q18.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------PhysicalProject ---------------PhysicalOlapScan[lineitem] -------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute ---------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------filter((sum(l_quantity) > 300.00)) ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[lineitem] +----------------PhysicalOlapScan[lineitem] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------filter((sum(l_quantity) > 300.00)) +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out index 150f569cdfc749..44f7c223f11375 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q19.out @@ -1,15 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) -----------PhysicalProject -------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) ---------------PhysicalOlapScan[lineitem] -----------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))(part.p_size >= 1)) -----------------PhysicalOlapScan[part] +--------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) +----------------PhysicalOlapScan[lineitem] +------------PhysicalDistribute +--------------PhysicalProject +----------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))(part.p_size >= 1)) +------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out index ec4c6370b694a4..e4513c0ef82bb4 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q2.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------PhysicalOlapScan[supplier] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[nation] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((region.r_name = 'EUROPE')) -----------------------------------PhysicalOlapScan[region] ---------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalProject -----------------------------filter((part.p_size = 15)(p_type like '%BRASS')) -------------------------------PhysicalOlapScan[part] +------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------PhysicalOlapScan[supplier] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((region.r_name = 'EUROPE')) +------------------------------------PhysicalOlapScan[region] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] +----------------------------PhysicalProject +------------------------------filter((part.p_size = 15)(p_type like '%BRASS')) +--------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out index 2e6728910560ae..300ef0d9d617ea 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20-rewrite.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = t3.ps_suppkey) -----------PhysicalDistribute -------------PhysicalProject ---------------hashJoin[INNER_JOIN](t2.l_partkey = t1.ps_partkey)(t2.l_suppkey = t1.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > t2.l_q) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute -------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] ---------------------PhysicalProject -----------------------filter((p_name like 'forest%')) -------------------------PhysicalOlapScan[part] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = t3.ps_suppkey) +------------PhysicalDistribute --------------PhysicalProject -----------------PhysicalOlapScan[supplier] ---------------PhysicalDistribute +----------------hashJoin[INNER_JOIN](t2.l_partkey = t1.ps_partkey)(t2.l_suppkey = t1.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > t2.l_q) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute +--------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] +----------------------PhysicalProject +------------------------filter((p_name like 'forest%')) +--------------------------PhysicalOlapScan[part] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------PhysicalProject -------------------filter((nation.n_name = 'CANADA')) ---------------------PhysicalOlapScan[nation] +------------------PhysicalOlapScan[supplier] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((nation.n_name = 'CANADA')) +----------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out index af742120ca219e..d3901ee9c271af 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q20.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) -----------PhysicalDistribute -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute -------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] ---------------------PhysicalProject -----------------------filter((p_name like 'forest%')) -------------------------PhysicalOlapScan[part] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) +------------PhysicalDistribute --------------PhysicalProject -----------------PhysicalOlapScan[supplier] ---------------PhysicalDistribute +----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute +--------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] +----------------------PhysicalProject +------------------------filter((p_name like 'forest%')) +--------------------------PhysicalOlapScan[part] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------PhysicalProject -------------------filter((nation.n_name = 'CANADA')) ---------------------PhysicalOlapScan[nation] +------------------PhysicalOlapScan[supplier] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((nation.n_name = 'CANADA')) +----------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out index 1ebbb4a8396361..2f4348619b2373 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q21.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) -----------------PhysicalProject -------------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject ----------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) ------------------PhysicalProject --------------------filter((orders.o_orderstatus = 'F')) ----------------------PhysicalOlapScan[orders] -------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +------------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) --------------------PhysicalProject -----------------------filter((l3.l_receiptdate > l3.l_commitdate)) -------------------------PhysicalOlapScan[lineitem] ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) ----------------------PhysicalProject -------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +------------------------filter((l3.l_receiptdate > l3.l_commitdate)) --------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[supplier] ---------------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +------------------------PhysicalProject +--------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +----------------------------PhysicalOlapScan[lineitem] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------------PhysicalProject -------------------------------filter((nation.n_name = 'SAUDI ARABIA')) ---------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalOlapScan[supplier] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((nation.n_name = 'SAUDI ARABIA')) +----------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out index d5088ac56b6452..3525bb716889f9 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q22.out @@ -8,20 +8,22 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[orders] -------------------PhysicalDistribute ---------------------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] +----------------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 -------------------------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] \ No newline at end of file +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +----------------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalAssertNumRows +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +--------------------------------PhysicalOlapScan[customer] + diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out index 23fc521663607a..da651e12cda6bc 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q3.out @@ -1,24 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ---------------PhysicalProject -----------------filter((lineitem.l_shipdate > 1995-03-15)) -------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-03-15)) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((customer.c_mktsegment = 'BUILDING')) ---------------------------PhysicalOlapScan[customer] +------------------filter((lineitem.l_shipdate > 1995-03-15)) +--------------------PhysicalOlapScan[lineitem] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((orders.o_orderdate < 1995-03-15)) +----------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((customer.c_mktsegment = 'BUILDING')) +----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out index 05c67b9b65c80b..4d9d91d96323ae 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q4.out @@ -1,17 +1,18 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------PhysicalProject -------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) ---------------------PhysicalOlapScan[lineitem] -----------------PhysicalProject -------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) ---------------------PhysicalOlapScan[orders] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------PhysicalProject +--------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) +----------------------PhysicalOlapScan[lineitem] +------------------PhysicalProject +--------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) +----------------------PhysicalOlapScan[orders] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out index f8211c8fa8639e..bf8fb1842de788 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q5.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineitem] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) ------------------------------------PhysicalProject ---------------------------------------filter((region.r_name = 'ASIA')) -----------------------------------------PhysicalOlapScan[region] -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) ---------------------------PhysicalOlapScan[orders] +--------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((region.r_name = 'ASIA')) +------------------------------------------PhysicalOlapScan[region] +------------------------PhysicalProject +--------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) +----------------------------PhysicalOlapScan[orders] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out index da938bd9de6bab..0cd7c1f4b2b411 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q6.out @@ -1,9 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) -----------PhysicalOlapScan[lineitem] +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) +------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out index 8969786cfaac9f..2eefbd562eb216 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q7.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ---------------------------PhysicalProject -----------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) -------------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) -----------------------------------PhysicalProject -------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) ---------------------------------------PhysicalOlapScan[nation] -----------------------------------PhysicalDistribute +--------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +----------------------------PhysicalProject +------------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) +--------------------------------PhysicalOlapScan[lineitem] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) ------------------------------------PhysicalProject ---------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) +--------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) ----------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) +------------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out index cd339810e07e96..07a9cd9ca20154 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q8.out @@ -1,48 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[supplier] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer] -------------------------------------PhysicalDistribute ---------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------------------------PhysicalProject -------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) ---------------------------------------------PhysicalOlapScan[orders] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[lineitem] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) ---------------------------------------------------PhysicalOlapScan[part] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[supplier] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) ------------------------------PhysicalProject ---------------------------------filter((region.r_name = 'AMERICA')) -----------------------------------PhysicalOlapScan[region] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[nation] +--------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[customer] +--------------------------------------PhysicalDistribute +----------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------------------------PhysicalProject +--------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) +----------------------------------------------PhysicalOlapScan[orders] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[lineitem] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) +----------------------------------------------------PhysicalOlapScan[part] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((region.r_name = 'AMERICA')) +------------------------------------PhysicalOlapScan[region] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out index e20942535d3417..25705c06d641d2 100644 --- a/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf1000_p0/shape/q9.out @@ -1,38 +1,39 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ---------------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +------------------------------PhysicalProject +--------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[lineitem] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------filter((p_name like '%green%')) +----------------------------------------PhysicalOlapScan[part] +------------------------------PhysicalDistribute +--------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[lineitem] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------filter((p_name like '%green%')) ---------------------------------------PhysicalOlapScan[part] -----------------------------PhysicalDistribute -------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[supplier] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] +------------------------------------PhysicalOlapScan[supplier] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[nation] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[partsupp] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out index 00a7cf50f83f52..7740edd76cc3a5 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q1.out @@ -1,12 +1,13 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------filter((lineitem.l_shipdate <= 1998-09-02)) -----------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------filter((lineitem.l_shipdate <= 1998-09-02)) +------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out index c1d1c3548aedb4..bbf108f0aedc48 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q10.out @@ -1,25 +1,29 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] ----------------PhysicalProject -------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------PhysicalProject -------------------------filter((lineitem.l_returnflag = 'R')) ---------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) ---------------------------PhysicalOlapScan[orders] ---------------PhysicalDistribute -----------------PhysicalProject -------------------PhysicalOlapScan[nation] +------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +--------------------PhysicalProject +----------------------filter((lineitem.l_returnflag = 'R')) +------------------------PhysicalOlapScan[lineitem] +--------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](customer.c_nationkey = nation.n_nationkey) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[customer] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((orders.o_orderdate < 1994-01-01)(orders.o_orderdate >= 1993-10-01)) +----------------------------------PhysicalOlapScan[orders] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out index cf27cdd93f4d44..914f6976ee3744 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q11.out @@ -1,39 +1,40 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * ps_availqty)) * 0.000002) as DOUBLE)) -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------PhysicalProject -------------------PhysicalOlapScan[partsupp] -----------------PhysicalDistribute -------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[supplier] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((nation.n_name = 'GERMANY')) ---------------------------PhysicalOlapScan[nation] -----------PhysicalDistribute -------------PhysicalAssertNumRows +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------NestedLoopJoin[INNER_JOIN](cast(value as DOUBLE) > cast((sum((ps_supplycost * ps_availqty)) * 0.000002) as DOUBLE)) +------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashAgg[GLOBAL] +----------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------PhysicalProject +--------------------PhysicalOlapScan[partsupp] ------------------PhysicalDistribute ---------------------hashAgg[LOCAL] +--------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute +------------------------PhysicalOlapScan[supplier] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((nation.n_name = 'GERMANY')) +----------------------------PhysicalOlapScan[nation] +------------PhysicalDistribute +--------------PhysicalAssertNumRows +----------------PhysicalProject +------------------hashAgg[GLOBAL] +--------------------PhysicalDistribute +----------------------hashAgg[LOCAL] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------------------------PhysicalProject -----------------------------------filter((nation.n_name = 'GERMANY')) -------------------------------------PhysicalOlapScan[nation] +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((nation.n_name = 'GERMANY')) +--------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out index cf83a4373e7a7d..1fec7c6597fd30 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q12.out @@ -1,16 +1,17 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------PhysicalProject -------------------PhysicalOlapScan[orders] -----------------PhysicalProject -------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) ---------------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------PhysicalProject +--------------------PhysicalOlapScan[orders] +------------------PhysicalProject +--------------------filter(l_shipmode IN ('MAIL', 'SHIP')(lineitem.l_shipdate < lineitem.l_commitdate)(lineitem.l_receiptdate < 1995-01-01)(lineitem.l_receiptdate >= 1994-01-01)(lineitem.l_commitdate < lineitem.l_receiptdate)) +----------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out index ae50570ecf54f2..ab3e4ce172b9db 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q13.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashAgg[LOCAL] -----------------PhysicalProject -------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter(( not (o_comment like '%special%requests%'))) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------PhysicalOlapScan[customer] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashAgg[LOCAL] +------------------PhysicalProject +--------------------hashJoin[RIGHT_OUTER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(( not (o_comment like '%special%requests%'))) +----------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out index 9ec9d4f3fa9854..747b102ab8889a 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q14.out @@ -1,15 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalProject ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) -------------PhysicalProject ---------------PhysicalOlapScan[part] -------------PhysicalDistribute +PhysicalResultSink +--PhysicalProject +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](lineitem.l_partkey = part.p_partkey) --------------PhysicalProject -----------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) -------------------PhysicalOlapScan[lineitem] +----------------PhysicalOlapScan[part] +--------------PhysicalDistribute +----------------PhysicalProject +------------------filter((lineitem.l_shipdate < 1995-10-01)(lineitem.l_shipdate >= 1995-09-01)) +--------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out index da9dd396bebe3d..4106594748a355 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q15.out @@ -1,30 +1,31 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) -----------PhysicalProject -------------PhysicalOlapScan[supplier] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) ---------------hashAgg[GLOBAL] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[INNER_JOIN](supplier.s_suppkey = revenue0.supplier_no) +------------PhysicalProject +--------------PhysicalOlapScan[supplier] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](revenue0.total_revenue = max(total_revenue)) +----------------hashAgg[GLOBAL] +------------------PhysicalDistribute +--------------------hashAgg[LOCAL] +----------------------PhysicalProject +------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +--------------------------PhysicalOlapScan[lineitem] ----------------PhysicalDistribute -------------------hashAgg[LOCAL] ---------------------PhysicalProject -----------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute -----------------PhysicalAssertNumRows -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------hashAgg[GLOBAL] -----------------------------PhysicalDistribute -------------------------------hashAgg[LOCAL] ---------------------------------PhysicalProject -----------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) -------------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalAssertNumRows +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------hashAgg[GLOBAL] +------------------------------PhysicalDistribute +--------------------------------hashAgg[LOCAL] +----------------------------------PhysicalProject +------------------------------------filter((lineitem.l_shipdate >= 1996-01-01)(lineitem.l_shipdate < 1996-04-01)) +--------------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out index d72afcf57daec7..21332b6f99dc89 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q16.out @@ -1,21 +1,22 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) -----------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] -------------------PhysicalProject ---------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) -----------------------PhysicalOlapScan[part] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------filter((s_comment like '%Customer%Complaints%')) -----------------------PhysicalOlapScan[supplier] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[LEFT_ANTI_JOIN](partsupp.ps_suppkey = supplier.s_suppkey) +------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +--------------------PhysicalProject +----------------------PhysicalOlapScan[partsupp] +--------------------PhysicalProject +----------------------filter(( not (p_type like 'MEDIUM POLISHED%'))( not (p_brand = 'Brand#45'))p_size IN (3, 9, 14, 19, 23, 36, 45, 49)) +------------------------PhysicalOlapScan[part] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------filter((s_comment like '%Customer%Complaints%')) +------------------------PhysicalOlapScan[supplier] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out index 90b6d8a1903660..65bad6d59809a4 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q17.out @@ -1,20 +1,21 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalProject ---hashAgg[GLOBAL] -----PhysicalDistribute -------hashAgg[LOCAL] ---------PhysicalProject -----------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) -------------PhysicalWindow ---------------PhysicalQuickSort -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalProject +----hashAgg[GLOBAL] +------PhysicalDistribute +--------hashAgg[LOCAL] +----------PhysicalProject +------------filter((cast(l_quantity as DECIMALV3(38, 5)) < (0.2 * avg(cast(l_quantity as DECIMALV3(17, 4))) OVER(PARTITION BY p_partkey)))) +--------------PhysicalWindow +----------------PhysicalQuickSort +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ------------------------PhysicalProject ---------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) -----------------------------PhysicalOlapScan[part] +--------------------------PhysicalOlapScan[lineitem] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------filter((part.p_container = 'MED BOX')(part.p_brand = 'Brand#23')) +------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out index e91c302aaa36cb..ae401ef1cd429f 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q18.out @@ -1,25 +1,26 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[LOCAL] ---------PhysicalProject -----------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -------------PhysicalProject ---------------PhysicalOlapScan[lineitem] -------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[LOCAL] +----------PhysicalProject +------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) --------------PhysicalProject -----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) -------------------PhysicalProject ---------------------PhysicalOlapScan[customer] -------------------PhysicalDistribute ---------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------filter((sum(l_quantity) > 300.00)) ---------------------------hashAgg[LOCAL] -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[lineitem] +----------------PhysicalOlapScan[lineitem] +--------------PhysicalDistribute +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] +--------------------PhysicalDistribute +----------------------hashJoin[LEFT_SEMI_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------filter((sum(l_quantity) > 300.00)) +----------------------------hashAgg[LOCAL] +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out index 150f569cdfc749..44f7c223f11375 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q19.out @@ -1,15 +1,16 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) -----------PhysicalProject -------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) ---------------PhysicalOlapScan[lineitem] -----------PhysicalDistribute +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey)((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND ((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00))) AND (part.p_size <= 5)) OR ((((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) AND (part.p_size <= 10))) OR ((((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00))) AND (part.p_size <= 15))) ------------PhysicalProject ---------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))(part.p_size >= 1)) -----------------PhysicalOlapScan[part] +--------------filter(l_shipmode IN ('AIR', 'AIR REG')(lineitem.l_shipinstruct = 'DELIVER IN PERSON')((((lineitem.l_quantity >= 1.00) AND (lineitem.l_quantity <= 11.00)) OR ((lineitem.l_quantity >= 10.00) AND (lineitem.l_quantity <= 20.00))) OR ((lineitem.l_quantity >= 20.00) AND (lineitem.l_quantity <= 30.00)))) +----------------PhysicalOlapScan[lineitem] +------------PhysicalDistribute +--------------PhysicalProject +----------------filter((((((part.p_brand = 'Brand#12') AND p_container IN ('SM CASE', 'SM BOX', 'SM PACK', 'SM PKG')) AND (part.p_size <= 5)) OR (((part.p_brand = 'Brand#23') AND p_container IN ('MED BAG', 'MED BOX', 'MED PKG', 'MED PACK')) AND (part.p_size <= 10))) OR (((part.p_brand = 'Brand#34') AND p_container IN ('LG CASE', 'LG BOX', 'LG PACK', 'LG PKG')) AND (part.p_size <= 15)))(part.p_size >= 1)) +------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out index ec4c6370b694a4..e4513c0ef82bb4 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q2.out @@ -1,32 +1,33 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) -----------PhysicalWindow -------------PhysicalQuickSort ---------------PhysicalDistribute -----------------PhysicalProject -------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) ---------------------PhysicalProject -----------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------PhysicalOlapScan[supplier] -------------------------PhysicalDistribute ---------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -----------------------------PhysicalProject -------------------------------PhysicalOlapScan[nation] -----------------------------PhysicalDistribute -------------------------------PhysicalProject ---------------------------------filter((region.r_name = 'EUROPE')) -----------------------------------PhysicalOlapScan[region] ---------------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------filter((partsupp.ps_supplycost = min(ps_supplycost) OVER(PARTITION BY p_partkey))) +------------PhysicalWindow +--------------PhysicalQuickSort +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) ----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[partsupp] ---------------------------PhysicalProject -----------------------------filter((part.p_size = 15)(p_type like '%BRASS')) -------------------------------PhysicalOlapScan[part] +------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------PhysicalOlapScan[supplier] +--------------------------PhysicalDistribute +----------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) +------------------------------PhysicalProject +--------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((region.r_name = 'EUROPE')) +------------------------------------PhysicalOlapScan[region] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](part.p_partkey = partsupp.ps_partkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[partsupp] +----------------------------PhysicalProject +------------------------------filter((part.p_size = 15)(p_type like '%BRASS')) +--------------------------------PhysicalOlapScan[part] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out index af742120ca219e..d3901ee9c271af 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q20.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) -----------PhysicalDistribute -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) -----------------PhysicalProject -------------------hashAgg[GLOBAL] ---------------------PhysicalDistribute -----------------------hashAgg[LOCAL] -------------------------PhysicalProject ---------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) -----------------------------PhysicalOlapScan[lineitem] -----------------PhysicalDistribute -------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) ---------------------PhysicalProject -----------------------PhysicalOlapScan[partsupp] ---------------------PhysicalProject -----------------------filter((p_name like 'forest%')) -------------------------PhysicalOlapScan[part] -----------PhysicalDistribute -------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashJoin[RIGHT_SEMI_JOIN](supplier.s_suppkey = partsupp.ps_suppkey) +------------PhysicalDistribute --------------PhysicalProject -----------------PhysicalOlapScan[supplier] ---------------PhysicalDistribute +----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_partkey = partsupp.ps_partkey)(lineitem.l_suppkey = partsupp.ps_suppkey)(cast(ps_availqty as DECIMALV3(38, 3)) > (0.5 * sum(l_quantity))) +------------------PhysicalProject +--------------------hashAgg[GLOBAL] +----------------------PhysicalDistribute +------------------------hashAgg[LOCAL] +--------------------------PhysicalProject +----------------------------filter((lineitem.l_shipdate < 1995-01-01)(lineitem.l_shipdate >= 1994-01-01)) +------------------------------PhysicalOlapScan[lineitem] +------------------PhysicalDistribute +--------------------hashJoin[LEFT_SEMI_JOIN](partsupp.ps_partkey = part.p_partkey) +----------------------PhysicalProject +------------------------PhysicalOlapScan[partsupp] +----------------------PhysicalProject +------------------------filter((p_name like 'forest%')) +--------------------------PhysicalOlapScan[part] +------------PhysicalDistribute +--------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------PhysicalProject -------------------filter((nation.n_name = 'CANADA')) ---------------------PhysicalOlapScan[nation] +------------------PhysicalOlapScan[supplier] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------filter((nation.n_name = 'CANADA')) +----------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out index 1ebbb4a8396361..2f4348619b2373 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q21.out @@ -1,33 +1,34 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) -----------------PhysicalProject -------------------PhysicalOlapScan[lineitem] +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject ----------------hashJoin[INNER_JOIN](orders.o_orderkey = l1.l_orderkey) ------------------PhysicalProject --------------------filter((orders.o_orderstatus = 'F')) ----------------------PhysicalOlapScan[orders] -------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) +------------------hashJoin[RIGHT_SEMI_JOIN](l2.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) --------------------PhysicalProject -----------------------filter((l3.l_receiptdate > l3.l_commitdate)) -------------------------PhysicalOlapScan[lineitem] ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +----------------------PhysicalOlapScan[lineitem] +--------------------hashJoin[RIGHT_ANTI_JOIN](l3.l_orderkey = l1.l_orderkey)( not (l_suppkey = l_suppkey)) ----------------------PhysicalProject -------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +------------------------filter((l3.l_receiptdate > l3.l_commitdate)) --------------------------PhysicalOlapScan[lineitem] -----------------------PhysicalDistribute -------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[supplier] ---------------------------PhysicalDistribute +----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = l1.l_suppkey) +------------------------PhysicalProject +--------------------------filter((l1.l_receiptdate > l1.l_commitdate)) +----------------------------PhysicalOlapScan[lineitem] +------------------------PhysicalDistribute +--------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) ----------------------------PhysicalProject -------------------------------filter((nation.n_name = 'SAUDI ARABIA')) ---------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalOlapScan[supplier] +----------------------------PhysicalDistribute +------------------------------PhysicalProject +--------------------------------filter((nation.n_name = 'SAUDI ARABIA')) +----------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out index d5088ac56b6452..3525bb716889f9 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q22.out @@ -8,20 +8,22 @@ PhysicalResultSink ----------PhysicalDistribute ------------hashAgg[LOCAL] --------------PhysicalProject -----------------hashJoin[RIGHT_ANTI_JOIN](orders.o_custkey = customer.c_custkey) -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[orders] -------------------PhysicalDistribute ---------------------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] +----------------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 -------------------------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] \ No newline at end of file +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter(substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +----------------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalAssertNumRows +----------------------hashAgg[GLOBAL] +------------------------PhysicalDistribute +--------------------------hashAgg[LOCAL] +----------------------------PhysicalProject +------------------------------filter((customer.c_acctbal > 0.00)substring(c_phone, 1, 2) IN ('13', '31', '23', '29', '30', '18', '17')) +--------------------------------PhysicalOlapScan[customer] + diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out index 23fc521663607a..da651e12cda6bc 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q3.out @@ -1,24 +1,25 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalTopN ---PhysicalDistribute -----PhysicalTopN -------PhysicalProject ---------hashAgg[LOCAL] -----------PhysicalProject -------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ---------------PhysicalProject -----------------filter((lineitem.l_shipdate > 1995-03-15)) -------------------PhysicalOlapScan[lineitem] ---------------PhysicalDistribute +PhysicalResultSink +--PhysicalTopN +----PhysicalDistribute +------PhysicalTopN +--------PhysicalProject +----------hashAgg[LOCAL] +------------PhysicalProject +--------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) ----------------PhysicalProject -------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-03-15)) ---------------------------PhysicalOlapScan[orders] ---------------------PhysicalDistribute -----------------------PhysicalProject -------------------------filter((customer.c_mktsegment = 'BUILDING')) ---------------------------PhysicalOlapScan[customer] +------------------filter((lineitem.l_shipdate > 1995-03-15)) +--------------------PhysicalOlapScan[lineitem] +----------------PhysicalDistribute +------------------PhysicalProject +--------------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey) +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((orders.o_orderdate < 1995-03-15)) +----------------------------PhysicalOlapScan[orders] +----------------------PhysicalDistribute +------------------------PhysicalProject +--------------------------filter((customer.c_mktsegment = 'BUILDING')) +----------------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out index 05c67b9b65c80b..4d9d91d96323ae 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q4.out @@ -1,17 +1,18 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------PhysicalProject -------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) ---------------------PhysicalOlapScan[lineitem] -----------------PhysicalProject -------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) ---------------------PhysicalOlapScan[orders] +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[RIGHT_SEMI_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------PhysicalProject +--------------------filter((lineitem.l_commitdate < lineitem.l_receiptdate)) +----------------------PhysicalOlapScan[lineitem] +------------------PhysicalProject +--------------------filter((orders.o_orderdate >= 1993-07-01)(orders.o_orderdate < 1993-10-01)) +----------------------PhysicalOlapScan[orders] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out index f8211c8fa8639e..bf8fb1842de788 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q5.out @@ -1,35 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) ---------------------------PhysicalProject -----------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = supplier.s_nationkey) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](lineitem.l_suppkey = supplier.s_suppkey) +----------------------------PhysicalProject +------------------------------PhysicalOlapScan[lineitem] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------hashJoin[INNER_JOIN](nation.n_regionkey = region.r_regionkey) ------------------------------------PhysicalProject ---------------------------------------filter((region.r_name = 'ASIA')) -----------------------------------------PhysicalOlapScan[region] -----------------------PhysicalProject -------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) ---------------------------PhysicalOlapScan[orders] +--------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter((region.r_name = 'ASIA')) +------------------------------------------PhysicalOlapScan[region] +------------------------PhysicalProject +--------------------------filter((orders.o_orderdate < 1995-01-01)(orders.o_orderdate >= 1994-01-01)) +----------------------------PhysicalOlapScan[orders] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[customer] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out index da938bd9de6bab..0cd7c1f4b2b411 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q6.out @@ -1,9 +1,10 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -hashAgg[GLOBAL] ---PhysicalDistribute -----hashAgg[LOCAL] -------PhysicalProject ---------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) -----------PhysicalOlapScan[lineitem] +PhysicalResultSink +--hashAgg[GLOBAL] +----PhysicalDistribute +------hashAgg[LOCAL] +--------PhysicalProject +----------filter((lineitem.l_shipdate >= 1994-01-01)(lineitem.l_discount <= 0.07)(lineitem.l_discount >= 0.05)(lineitem.l_quantity < 24.00)(lineitem.l_shipdate < 1995-01-01)) +------------PhysicalOlapScan[lineitem] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out index 8969786cfaac9f..2eefbd562eb216 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q7.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) -----------------PhysicalProject -------------------PhysicalOlapScan[customer] -----------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](customer.c_custkey = orders.o_custkey)(customer.c_nationkey = n2.n_nationkey) ------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ---------------------------PhysicalProject -----------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) -------------------------------PhysicalOlapScan[lineitem] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute ---------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) -----------------------------------PhysicalProject -------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) ---------------------------------------PhysicalOlapScan[nation] -----------------------------------PhysicalDistribute +--------------------PhysicalOlapScan[customer] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +----------------------------PhysicalProject +------------------------------filter((lineitem.l_shipdate <= 1996-12-31)(lineitem.l_shipdate >= 1995-01-01)) +--------------------------------PhysicalOlapScan[lineitem] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n1.n_nationkey) +--------------------------------PhysicalProject +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------NestedLoopJoin[INNER_JOIN](((n1.n_name = 'FRANCE') AND (n2.n_name = 'GERMANY')) OR ((n1.n_name = 'GERMANY') AND (n2.n_name = 'FRANCE'))) ------------------------------------PhysicalProject ---------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) +--------------------------------------filter(((n1.n_name = 'FRANCE') OR (n1.n_name = 'GERMANY'))) ----------------------------------------PhysicalOlapScan[nation] +------------------------------------PhysicalDistribute +--------------------------------------PhysicalProject +----------------------------------------filter(((n2.n_name = 'GERMANY') OR (n2.n_name = 'FRANCE'))) +------------------------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out index cd339810e07e96..07a9cd9ca20154 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q8.out @@ -1,48 +1,49 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------PhysicalProject ---------hashAgg[GLOBAL] -----------PhysicalDistribute -------------hashAgg[LOCAL] ---------------PhysicalProject -----------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[supplier] -----------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------PhysicalProject +----------hashAgg[GLOBAL] +------------PhysicalDistribute +--------------hashAgg[LOCAL] +----------------PhysicalProject +------------------hashJoin[INNER_JOIN](supplier.s_nationkey = n2.n_nationkey) +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ------------------------PhysicalProject ---------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) -----------------------------PhysicalProject -------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) ---------------------------------PhysicalProject -----------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) -------------------------------------PhysicalProject ---------------------------------------PhysicalOlapScan[customer] -------------------------------------PhysicalDistribute ---------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) -----------------------------------------PhysicalProject -------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) ---------------------------------------------PhysicalOlapScan[orders] -----------------------------------------PhysicalProject -------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) ---------------------------------------------PhysicalProject -----------------------------------------------PhysicalOlapScan[lineitem] ---------------------------------------------PhysicalDistribute -----------------------------------------------PhysicalProject -------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) ---------------------------------------------------PhysicalOlapScan[part] ---------------------------------PhysicalDistribute -----------------------------------PhysicalProject -------------------------------------PhysicalOlapScan[nation] -----------------------------PhysicalDistribute +--------------------------PhysicalOlapScan[supplier] +------------------------PhysicalDistribute +--------------------------PhysicalProject +----------------------------hashJoin[INNER_JOIN](n1.n_regionkey = region.r_regionkey) ------------------------------PhysicalProject ---------------------------------filter((region.r_name = 'AMERICA')) -----------------------------------PhysicalOlapScan[region] -------------------PhysicalDistribute ---------------------PhysicalProject -----------------------PhysicalOlapScan[nation] +--------------------------------hashJoin[INNER_JOIN](customer.c_nationkey = n1.n_nationkey) +----------------------------------PhysicalProject +------------------------------------hashJoin[INNER_JOIN](orders.o_custkey = customer.c_custkey) +--------------------------------------PhysicalProject +----------------------------------------PhysicalOlapScan[customer] +--------------------------------------PhysicalDistribute +----------------------------------------hashJoin[INNER_JOIN](lineitem.l_orderkey = orders.o_orderkey) +------------------------------------------PhysicalProject +--------------------------------------------filter((orders.o_orderdate <= 1996-12-31)(orders.o_orderdate >= 1995-01-01)) +----------------------------------------------PhysicalOlapScan[orders] +------------------------------------------PhysicalProject +--------------------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) +----------------------------------------------PhysicalProject +------------------------------------------------PhysicalOlapScan[lineitem] +----------------------------------------------PhysicalDistribute +------------------------------------------------PhysicalProject +--------------------------------------------------filter((part.p_type = 'ECONOMY ANODIZED STEEL')) +----------------------------------------------------PhysicalOlapScan[part] +----------------------------------PhysicalDistribute +------------------------------------PhysicalProject +--------------------------------------PhysicalOlapScan[nation] +------------------------------PhysicalDistribute +--------------------------------PhysicalProject +----------------------------------filter((region.r_name = 'AMERICA')) +------------------------------------PhysicalOlapScan[region] +--------------------PhysicalDistribute +----------------------PhysicalProject +------------------------PhysicalOlapScan[nation] diff --git a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out index 6a73f8f1f00996..a46d07b12562dc 100644 --- a/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out +++ b/regression-test/data/nereids_tpch_shape_sf500_p0/shape/q9.out @@ -1,36 +1,37 @@ -- This file is automatically generated. You should know what you did if you want to edit this -- !select -- -PhysicalQuickSort ---PhysicalDistribute -----PhysicalQuickSort -------hashAgg[GLOBAL] ---------PhysicalDistribute -----------hashAgg[LOCAL] -------------PhysicalProject ---------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) -----------------PhysicalDistribute -------------------PhysicalProject ---------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) -----------------------PhysicalProject -------------------------PhysicalOlapScan[orders] -----------------------PhysicalProject -------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) ---------------------------PhysicalProject -----------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[lineitem] -------------------------------PhysicalDistribute +PhysicalResultSink +--PhysicalQuickSort +----PhysicalDistribute +------PhysicalQuickSort +--------hashAgg[GLOBAL] +----------PhysicalDistribute +------------hashAgg[LOCAL] +--------------PhysicalProject +----------------hashJoin[INNER_JOIN](partsupp.ps_partkey = lineitem.l_partkey)(partsupp.ps_suppkey = lineitem.l_suppkey) +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------hashJoin[INNER_JOIN](orders.o_orderkey = lineitem.l_orderkey) +------------------------PhysicalProject +--------------------------PhysicalOlapScan[orders] +------------------------PhysicalProject +--------------------------hashJoin[INNER_JOIN](supplier.s_suppkey = lineitem.l_suppkey) +----------------------------PhysicalProject +------------------------------hashJoin[INNER_JOIN](part.p_partkey = lineitem.l_partkey) --------------------------------PhysicalProject -----------------------------------filter((p_name like '%green%')) -------------------------------------PhysicalOlapScan[part] ---------------------------PhysicalDistribute -----------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) -------------------------------PhysicalProject ---------------------------------PhysicalOlapScan[supplier] -------------------------------PhysicalDistribute +----------------------------------PhysicalOlapScan[lineitem] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------filter((p_name like '%green%')) +--------------------------------------PhysicalOlapScan[part] +----------------------------PhysicalDistribute +------------------------------hashJoin[INNER_JOIN](supplier.s_nationkey = nation.n_nationkey) --------------------------------PhysicalProject -----------------------------------PhysicalOlapScan[nation] -----------------PhysicalDistribute -------------------PhysicalProject ---------------------PhysicalOlapScan[partsupp] +----------------------------------PhysicalOlapScan[supplier] +--------------------------------PhysicalDistribute +----------------------------------PhysicalProject +------------------------------------PhysicalOlapScan[nation] +------------------PhysicalDistribute +--------------------PhysicalProject +----------------------PhysicalOlapScan[partsupp] From 01a2b30fc4b00c4ddc2b69ecc7a2771a39b2de31 Mon Sep 17 00:00:00 2001 From: lichi <12095047@qq.com> Date: Thu, 3 Aug 2023 20:02:43 +0800 Subject: [PATCH 8/8] [fix](nereids)pick several pr for bug fix --- .../apache/doris/nereids/CascadesContext.java | 3 +-- .../doris/nereids/jobs/executor/Rewriter.java | 4 +--- .../nereids/rules/analysis/BindRelation.java | 7 +++++-- .../rules/analysis/UserAuthentication.java | 9 ++++----- .../trees/plans/logical/LogicalAggregate.java | 2 +- .../trees/plans/logical/LogicalJdbcScan.java | 3 +-- .../plans/physical/PhysicalCTEProducer.java | 3 ++- .../plans/physical/PhysicalJdbcScan.java | 19 ++++++++----------- .../nereids/trees/plans/PlanOutputTest.java | 2 -- 9 files changed, 23 insertions(+), 29 deletions(-) diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java index dab6377addf0f8..872a28571b2ef0 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/CascadesContext.java @@ -20,9 +20,8 @@ import org.apache.doris.catalog.DatabaseIf; import org.apache.doris.catalog.Env; import org.apache.doris.catalog.TableIf; -import org.apache.doris.datasource.CatalogIf; -import org.apache.doris.catalog.Table; import org.apache.doris.common.Pair; +import org.apache.doris.datasource.CatalogIf; import org.apache.doris.nereids.analyzer.Scope; import org.apache.doris.nereids.analyzer.UnboundRelation; import org.apache.doris.nereids.jobs.Job; 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 16fbb6979b733b..06ba235c285f8d 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 @@ -64,8 +64,6 @@ import org.apache.doris.nereids.rules.rewrite.InferFilterNotNull; import org.apache.doris.nereids.rules.rewrite.InferJoinNotNull; import org.apache.doris.nereids.rules.rewrite.InferPredicates; -import org.apache.doris.nereids.rules.rewrite.InlineCTE; -import org.apache.doris.nereids.rules.rewrite.InferSetOperatorDistinct; import org.apache.doris.nereids.rules.rewrite.MergeFilters; import org.apache.doris.nereids.rules.rewrite.MergeOneRowRelationIntoUnion; import org.apache.doris.nereids.rules.rewrite.MergeProjects; @@ -75,8 +73,8 @@ import org.apache.doris.nereids.rules.rewrite.PruneFileScanPartition; import org.apache.doris.nereids.rules.rewrite.PruneOlapScanPartition; import org.apache.doris.nereids.rules.rewrite.PruneOlapScanTablet; -import org.apache.doris.nereids.rules.rewrite.PushConjunctsIntoJdbcScan; import org.apache.doris.nereids.rules.rewrite.PullUpCteAnchor; +import org.apache.doris.nereids.rules.rewrite.PushConjunctsIntoJdbcScan; import org.apache.doris.nereids.rules.rewrite.PushFilterInsideJoin; import org.apache.doris.nereids.rules.rewrite.PushProjectIntoOneRowRelation; import org.apache.doris.nereids.rules.rewrite.PushProjectThroughUnion; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java index 2a68fcd215d00a..425641fda3a0ac 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindRelation.java @@ -42,6 +42,7 @@ import org.apache.doris.nereids.trees.expressions.EqualTo; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PreAggStatus; @@ -214,11 +215,13 @@ private LogicalPlan getLogicalPlan(TableIf table, UnboundRelation unboundRelatio return new LogicalSubQueryAlias<>(tableQualifier, hiveViewPlan); } } - return new LogicalFileScan(RelationUtil.newRelationId(), (HMSExternalTable) table, tableQualifier); + return new LogicalFileScan(StatementScopeIdGenerator.newRelationId(), + (HMSExternalTable) table, tableQualifier); case ICEBERG_EXTERNAL_TABLE: case PAIMON_EXTERNAL_TABLE: case MAX_COMPUTE_EXTERNAL_TABLE: - return new LogicalFileScan(RelationUtil.newRelationId(), (ExternalTable) table, tableQualifier); + return new LogicalFileScan(StatementScopeIdGenerator.newRelationId(), + (ExternalTable) table, tableQualifier); case SCHEMA: return new LogicalSchemaScan(unboundRelation.getRelationId(), table, ImmutableList.of(dbName)); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/UserAuthentication.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/UserAuthentication.java index 17e32fa95aed89..f82619fb739f1f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/UserAuthentication.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/UserAuthentication.java @@ -26,7 +26,6 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.algebra.CatalogRelation; import org.apache.doris.qe.ConnectContext; @@ -50,18 +49,18 @@ private Plan checkPermission(CatalogRelation relation, ConnectContext connectCon } TableIf table = relation.getTable(); if (table == null) { - return relation; + return null; } String tableName = table.getName(); DatabaseIf db = table.getDatabase(); // when table inatanceof FunctionGenTable,db will be null if (db == null) { - return relation; + return null; } String dbName = db.getFullName(); CatalogIf catalog = db.getCatalog(); if (catalog == null) { - return relation; + return null; } String ctlName = catalog.getName(); // TODO: 2023/7/19 checkColumnsPriv @@ -72,6 +71,6 @@ private Plan checkPermission(CatalogRelation relation, ConnectContext connectCon ctlName + ": " + dbName + ": " + tableName); throw new AnalysisException(message); } - return relation; + return null; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java index 6672f9e81aaf5e..38f2b161900893 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalAggregate.java @@ -112,7 +112,7 @@ public LogicalAggregate( List outputExpressions, boolean normalized, boolean ordinalIsResolved, - Optional sourceRepeat, + Optional> sourceRepeat, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJdbcScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJdbcScan.java index 8b3df69d16cf04..c86bd0fb7dfffd 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJdbcScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalJdbcScan.java @@ -23,7 +23,6 @@ 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.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.RelationId; @@ -58,7 +57,7 @@ public LogicalJdbcScan(RelationId id, TableIf table, List qualifier, } public LogicalJdbcScan(RelationId id, TableIf table, List qualifier) { - this(id, table, qualifier, Optional.empty(), Optional.empty(),ImmutableSet.of()); + this(id, table, qualifier, Optional.empty(), Optional.empty(), ImmutableSet.of()); } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java index a3917dafc9551b..8614aa26d8f5d5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalCTEProducer.java @@ -22,6 +22,7 @@ import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.CTEId; import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; @@ -136,7 +137,7 @@ public List computeOutput() { @Override public PhysicalCTEProducer resetLogicalProperties() { - return new PhysicalCTEProducer<>(cteId, projects, groupExpression, null, physicalProperties, + return new PhysicalCTEProducer<>(cteId, groupExpression, null, physicalProperties, statistics, child()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalJdbcScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalJdbcScan.java index d91d2b22f6c027..a91dba2ee5bbca 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalJdbcScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalJdbcScan.java @@ -22,7 +22,6 @@ import org.apache.doris.nereids.properties.LogicalProperties; import org.apache.doris.nereids.properties.PhysicalProperties; import org.apache.doris.nereids.trees.expressions.Expression; -import org.apache.doris.nereids.trees.plans.ObjectId; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.RelationId; @@ -33,6 +32,7 @@ import com.google.common.collect.ImmutableSet; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.Set; @@ -42,17 +42,15 @@ public class PhysicalJdbcScan extends PhysicalCatalogRelation { private final TableIf table; - private final DistributionSpec distributionSpec; private final Set conjuncts; /** * Constructor for PhysicalJdbcScan. */ - public PhysicalJdbcScan(RelationId id, TableIf table, List qualifier, DistributionSpec distributionSpec, + public PhysicalJdbcScan(RelationId id, TableIf table, List qualifier, Optional groupExpression, LogicalProperties logicalProperties, Set conjuncts) { - super(id, PlanType.PHYSICAL_JDBC_SCAN, qualifier, groupExpression, logicalProperties); + super(id, PlanType.PHYSICAL_JDBC_SCAN, table, qualifier, groupExpression, logicalProperties); this.table = table; - this.distributionSpec = distributionSpec; this.conjuncts = ImmutableSet.copyOf(Objects.requireNonNull(conjuncts, "conjuncts should not be null")); } @@ -60,13 +58,12 @@ public PhysicalJdbcScan(RelationId id, TableIf table, List qualifier, Di * Constructor for PhysicalJdbcScan. */ public PhysicalJdbcScan(RelationId id, TableIf table, List qualifier, - DistributionSpec distributionSpec, Optional groupExpression, + Optional groupExpression, LogicalProperties logicalProperties, PhysicalProperties physicalProperties, Statistics statistics, Set conjuncts) { - super(id, PlanType.PHYSICAL_JDBC_SCAN, qualifier, groupExpression, + super(id, PlanType.PHYSICAL_JDBC_SCAN, table, qualifier, groupExpression, logicalProperties, physicalProperties, statistics); this.table = table; - this.distributionSpec = distributionSpec; this.conjuncts = ImmutableSet.copyOf(Objects.requireNonNull(conjuncts, "conjuncts should not be null")); } @@ -86,14 +83,14 @@ public R accept(PlanVisitor visitor, C context) { @Override public PhysicalJdbcScan withGroupExpression(Optional groupExpression) { - return new PhysicalJdbcScan(relationId, table, qualifier, distributionSpec, + return new PhysicalJdbcScan(relationId, table, qualifier, groupExpression, getLogicalProperties(), conjuncts); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalJdbcScan(relationId, table, qualifier, distributionSpec, + return new PhysicalJdbcScan(relationId, table, qualifier, groupExpression, logicalProperties.get(), conjuncts); } @@ -105,7 +102,7 @@ public TableIf getTable() { @Override public PhysicalJdbcScan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalJdbcScan(relationId, table, qualifier, distributionSpec, groupExpression, + return new PhysicalJdbcScan(relationId, table, qualifier, groupExpression, getLogicalProperties(), physicalProperties, statistics, conjuncts); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanOutputTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanOutputTest.java index 65694a18aedbde..0bef6c66a4e382 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanOutputTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/PlanOutputTest.java @@ -25,8 +25,6 @@ import org.apache.doris.nereids.types.IntegerType; import org.apache.doris.nereids.types.StringType; import org.apache.doris.nereids.util.PlanConstructor; -import org.apache.doris.nereids.util.RelationUtil; -import org.apache.doris.statistics.Statistics; import com.google.common.collect.ImmutableList; import org.junit.jupiter.api.Assertions;