diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/DescriptorTable.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/DescriptorTable.java index d793c4f70c52d8..254edea5fbbc89 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/DescriptorTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/DescriptorTable.java @@ -20,7 +20,6 @@ package org.apache.doris.analysis; -import org.apache.doris.catalog.RecursiveCteTempTable; import org.apache.doris.catalog.TableIf; import org.apache.doris.common.IdGenerator; import org.apache.doris.thrift.TDescriptorTable; @@ -101,10 +100,6 @@ public TDescriptorTable toThrift() { } for (TableIf tbl : referencedTbls.values()) { - if (tbl instanceof RecursiveCteTempTable) { - // skip recursive cte temp table - continue; - } result.addToTableDescriptors(tbl.toThrift()); } thriftDescTable = result; diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/RecursiveCteTempTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/RecursiveCteTempTable.java deleted file mode 100644 index 54c819d45dd5dd..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/RecursiveCteTempTable.java +++ /dev/null @@ -1,35 +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.catalog; - -import org.apache.doris.common.SystemIdGenerator; - -import com.google.common.collect.ImmutableList; - -import java.util.List; - -public class RecursiveCteTempTable extends Table { - public RecursiveCteTempTable(String tableName, List fullSchema) { - super(SystemIdGenerator.getNextId(), tableName, TableType.RECURSIVE_CTE_TEMP_TABLE, fullSchema); - } - - @Override - public List getFullQualifiers() { - return ImmutableList.of(name); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java index b3241a43598287..9a71df0a76a073 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/TableIf.java @@ -450,8 +450,7 @@ enum TableType { HUDI, JDBC, TABLE_VALUED_FUNCTION, HMS_EXTERNAL_TABLE, ES_EXTERNAL_TABLE, MATERIALIZED_VIEW, JDBC_EXTERNAL_TABLE, ICEBERG_EXTERNAL_TABLE, TEST_EXTERNAL_TABLE, PAIMON_EXTERNAL_TABLE, MAX_COMPUTE_EXTERNAL_TABLE, - HUDI_EXTERNAL_TABLE, TRINO_CONNECTOR_EXTERNAL_TABLE, LAKESOUl_EXTERNAL_TABLE, DICTIONARY, DORIS_EXTERNAL_TABLE, - RECURSIVE_CTE_TEMP_TABLE; + HUDI_EXTERNAL_TABLE, TRINO_CONNECTOR_EXTERNAL_TABLE, LAKESOUl_EXTERNAL_TABLE, DICTIONARY, DORIS_EXTERNAL_TABLE; public String toEngineName() { switch (this) { @@ -494,8 +493,6 @@ public String toEngineName() { return "dictionary"; case DORIS_EXTERNAL_TABLE: return "External_Doris"; - case RECURSIVE_CTE_TEMP_TABLE: - return "RecursiveCteTempTable"; default: return null; } @@ -535,7 +532,6 @@ public String toMysqlType() { case MATERIALIZED_VIEW: case TRINO_CONNECTOR_EXTERNAL_TABLE: case DORIS_EXTERNAL_TABLE: - case RECURSIVE_CTE_TEMP_TABLE: return "BASE TABLE"; default: return null; 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 f2266133b31baf..87d57a2c6b4819 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 @@ -19,13 +19,16 @@ import org.apache.doris.nereids.exceptions.AnalysisException; 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.LogicalPlan; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; import org.apache.doris.qe.GlobalVariable; +import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import java.util.List; import java.util.Locale; import java.util.Map; import java.util.Optional; @@ -38,6 +41,7 @@ public class CTEContext { private final CTEId cteId; private final String name; + private List recursiveCteOutputs; // this cache only use once private LogicalPlan analyzedPlan; @@ -45,7 +49,7 @@ public class CTEContext { /* build head CTEContext */ public CTEContext() { - this(CTEId.DEFAULT, null, null); + this(CTEId.DEFAULT, null, (CTEContext) null); } /** @@ -66,6 +70,26 @@ public CTEContext(CTEId cteId, @Nullable LogicalSubQueryAlias parsedPlan, // if inner name same with outer name, use inner name in this scope. .buildKeepingLast(); this.cteId = cteId; + this.recursiveCteOutputs = ImmutableList.of(); + } + + /** + * CTEContext for recursive cte + */ + public CTEContext(CTEId cteId, String cteName, List recursiveCteOutputs) { + this.cteId = cteId; + this.name = GlobalVariable.lowerCaseTableNames != 0 ? cteName.toLowerCase(Locale.ROOT) : cteName; + this.recursiveCteOutputs = recursiveCteOutputs != null ? ImmutableList.copyOf(recursiveCteOutputs) + : ImmutableList.of(); + this.cteContextMap = ImmutableMap.of(name, this); + } + + public void setRecursiveCteOutputs(List recursiveCteOutputs) { + this.recursiveCteOutputs = recursiveCteOutputs; + } + + public List getRecursiveCteOutputs() { + return recursiveCteOutputs; } public void setAnalyzedPlan(LogicalPlan analyzedPlan) { 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 1517d6f2a7a52c..af108aafea3d1b 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 @@ -133,8 +133,7 @@ public class CascadesContext implements ScheduleContext { private final boolean isEnableExprTrace; private int groupExpressionCount = 0; - private Optional currentRecursiveCteName; - private List recursiveCteOutputs; + private Optional recursiveCteContext; /** * Constructor of OptimizerContext. @@ -145,7 +144,7 @@ public class CascadesContext implements ScheduleContext { private CascadesContext(Optional parent, Optional currentTree, StatementContext statementContext, Plan plan, Memo memo, CTEContext cteContext, PhysicalProperties requireProperties, boolean isLeadingDisableJoinReorder, - Optional currentRecursiveCteName, List recursiveCteOutputs) { + CTEContext recursiveCteContext) { 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"); @@ -170,8 +169,7 @@ private CascadesContext(Optional parent, Optional curren this.isEnableExprTrace = false; } this.isLeadingDisableJoinReorder = isLeadingDisableJoinReorder; - this.currentRecursiveCteName = currentRecursiveCteName; - this.recursiveCteOutputs = recursiveCteOutputs; + this.recursiveCteContext = Optional.ofNullable(recursiveCteContext); } /** init a temporary context to rewrite expression */ @@ -186,7 +184,7 @@ public static CascadesContext initTempContext() { } return newContext(Optional.empty(), Optional.empty(), statementContext, DUMMY_PLAN, - new CTEContext(), PhysicalProperties.ANY, false, Optional.empty(), ImmutableList.of()); + new CTEContext(), PhysicalProperties.ANY, false, null); } /** @@ -195,25 +193,23 @@ public static CascadesContext initTempContext() { public static CascadesContext initContext(StatementContext statementContext, Plan initPlan, PhysicalProperties requireProperties) { return newContext(Optional.empty(), Optional.empty(), statementContext, - initPlan, new CTEContext(), requireProperties, false, Optional.empty(), ImmutableList.of()); + initPlan, new CTEContext(), requireProperties, false, null); } /** * 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, Optional currentRecursiveCteName, - List recursiveCteOutputs) { + Plan initPlan, CTEContext cteContext, CTEContext recursiveCteContext) { return newContext(Optional.of(cascadesContext), Optional.empty(), cascadesContext.getStatementContext(), initPlan, cteContext, PhysicalProperties.ANY, - cascadesContext.isLeadingDisableJoinReorder, currentRecursiveCteName, recursiveCteOutputs); + cascadesContext.isLeadingDisableJoinReorder, recursiveCteContext); } public static CascadesContext newCurrentTreeContext(CascadesContext context) { return CascadesContext.newContext(context.getParent(), context.getCurrentTree(), context.getStatementContext(), context.getRewritePlan(), context.getCteContext(), - context.getCurrentJobContext().getRequiredProperties(), context.isLeadingDisableJoinReorder, - Optional.empty(), ImmutableList.of()); + context.getCurrentJobContext().getRequiredProperties(), context.isLeadingDisableJoinReorder, null); } /** @@ -222,17 +218,15 @@ public static CascadesContext newCurrentTreeContext(CascadesContext context) { public static CascadesContext newSubtreeContext(Optional subtree, CascadesContext context, Plan plan, PhysicalProperties requireProperties) { return CascadesContext.newContext(Optional.of(context), subtree, context.getStatementContext(), - plan, context.getCteContext(), requireProperties, context.isLeadingDisableJoinReorder, Optional.empty(), - ImmutableList.of()); + plan, context.getCteContext(), requireProperties, context.isLeadingDisableJoinReorder, null); } private static CascadesContext newContext(Optional parent, Optional subtree, StatementContext statementContext, Plan initPlan, CTEContext cteContext, PhysicalProperties requireProperties, boolean isLeadingDisableJoinReorder, - Optional currentRecursiveCteName, List recursiveCteOutputs) { + CTEContext recursiveCteContext) { return new CascadesContext(parent, subtree, statementContext, initPlan, null, - cteContext, requireProperties, isLeadingDisableJoinReorder, currentRecursiveCteName, - recursiveCteOutputs); + cteContext, requireProperties, isLeadingDisableJoinReorder, recursiveCteContext); } public CascadesContext getRoot() { @@ -259,16 +253,17 @@ public synchronized boolean isTimeout() { return isTimeout; } - public Optional getCurrentRecursiveCteName() { - return currentRecursiveCteName; + public Optional getRecursiveCteContext() { + return recursiveCteContext; } public List getRecursiveCteOutputs() { - return recursiveCteOutputs; + return recursiveCteContext.isPresent() ? recursiveCteContext.get().getRecursiveCteOutputs() + : ImmutableList.of(); } public boolean isAnalyzingRecursiveCteAnchorChild() { - return currentRecursiveCteName.isPresent() && recursiveCteOutputs.isEmpty(); + return recursiveCteContext.isPresent() && recursiveCteContext.get().getRecursiveCteOutputs().isEmpty(); } /** 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 93a7af2d8e095a..f72229451ce63b 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 @@ -291,6 +291,8 @@ public enum TableFrom { private List icebergRewriteFileScanTasks = null; private boolean hasNestedColumns; + private final Set mustInlineCTE = new HashSet<>(); + public StatementContext() { this(ConnectContext.get(), null, 0); } @@ -1057,4 +1059,12 @@ public boolean hasNestedColumns() { public void setHasNestedColumns(boolean hasNestedColumns) { this.hasNestedColumns = hasNestedColumns; } + + public void addToMustLineCTEs(CTEId cteId) { + mustInlineCTE.add(cteId); + } + + public Set getMustInlineCTEs() { + return mustInlineCTE; + } } 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 776ec163df27b5..0611e4db1faeb1 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 @@ -153,9 +153,9 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan; 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.PhysicalRecursiveCte; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteRecursiveChild; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnion; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionAnchor; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionProducer; 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.PhysicalResultSink; @@ -166,6 +166,7 @@ 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; +import org.apache.doris.nereids.trees.plans.physical.PhysicalWorkTableReference; import org.apache.doris.nereids.trees.plans.physical.RuntimeFilter; import org.apache.doris.nereids.trees.plans.physical.TopnFilter; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; @@ -1043,24 +1044,18 @@ public PlanFragment visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, PlanT } @Override - public PlanFragment visitPhysicalRecursiveCteScan(PhysicalRecursiveCteScan recursiveCteScan, + public PlanFragment visitPhysicalWorkTableReference(PhysicalWorkTableReference workTableReference, PlanTranslatorContext context) { - TableIf table = recursiveCteScan.getTable(); - List slots = ImmutableList.copyOf(recursiveCteScan.getOutput()); + List slots = ImmutableList.copyOf(workTableReference.getOutput()); TupleDescriptor tupleDescriptor = generateTupleDesc(slots, null, context); - - RecursiveCteScanNode scanNode = new RecursiveCteScanNode(table != null ? table.getName() : "", + RecursiveCteScanNode scanNode = new RecursiveCteScanNode(workTableReference.getTableName(), context.nextPlanNodeId(), tupleDescriptor); - scanNode.setNereidsId(recursiveCteScan.getId()); - context.getNereidsIdToPlanNodeIdMap().put(recursiveCteScan.getId(), scanNode.getId()); - Utils.execWithUncheckedException(scanNode::initScanRangeLocations); - - translateRuntimeFilter(recursiveCteScan, scanNode, context); + scanNode.setNereidsId(workTableReference.getId()); + context.getNereidsIdToPlanNodeIdMap().put(workTableReference.getId(), scanNode.getId()); - context.addScanNode(scanNode, recursiveCteScan); - PlanFragment planFragment = createPlanFragment(scanNode, DataPartition.RANDOM, recursiveCteScan); + PlanFragment planFragment = createPlanFragment(scanNode, DataPartition.RANDOM, workTableReference); context.addPlanFragment(planFragment); - updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), recursiveCteScan); + updateLegacyPlanIdToPhysicalPlan(planFragment.getPlanRoot(), workTableReference); return planFragment; } @@ -2192,10 +2187,8 @@ public PlanFragment visitPhysicalProject(PhysicalProject project if (inputPlanNode instanceof OlapScanNode) { ((OlapScanNode) inputPlanNode).updateRequiredSlots(context, requiredByProjectSlotIdSet); } - if (!(inputPlanNode instanceof RecursiveCteScanNode)) { - updateScanSlotsMaterialization((ScanNode) inputPlanNode, requiredSlotIdSet, - requiredByProjectSlotIdSet, context); - } + updateScanSlotsMaterialization((ScanNode) inputPlanNode, requiredSlotIdSet, + requiredByProjectSlotIdSet, context); } else { if (project.child() instanceof PhysicalDeferMaterializeTopN) { inputFragment.setOutputExprs(allProjectionExprs); @@ -2209,63 +2202,45 @@ public PlanFragment visitPhysicalProject(PhysicalProject project } @Override - public PlanFragment visitPhysicalRecursiveCte(PhysicalRecursiveCte recursiveCte, PlanTranslatorContext context) { + public PlanFragment visitPhysicalRecursiveUnion(PhysicalRecursiveUnion recursiveCte, + PlanTranslatorContext context) { List childrenFragments = new ArrayList<>(); for (Plan plan : recursiveCte.children()) { childrenFragments.add(plan.accept(this, context)); } - + List> distributeExprLists = getDistributeExprs(recursiveCte.children().toArray(new Plan[0])); TupleDescriptor setTuple = generateTupleDesc(recursiveCte.getOutput(), null, context); - List outputSlotDescs = new ArrayList<>(setTuple.getSlots()); RecursiveCteNode recursiveCteNode = new RecursiveCteNode(context.nextPlanNodeId(), setTuple.getId(), - recursiveCte.getCteName(), recursiveCte.isUnionAll()); - List> distributeExprLists = getDistributeExprs(recursiveCte.children().toArray(new Plan[0])); + recursiveCte.getCteName(), recursiveCte.isUnionAll()); + recursiveCteNode.setChildrenDistributeExprLists(distributeExprLists); recursiveCteNode.setNereidsId(recursiveCte.getId()); - List> resultExpressionLists = Lists.newArrayList(); context.getNereidsIdToPlanNodeIdMap().put(recursiveCte.getId(), recursiveCteNode.getId()); - for (List regularChildrenOutput : recursiveCte.getRegularChildrenOutputs()) { - resultExpressionLists.add(new ArrayList<>(regularChildrenOutput)); - } - for (PlanFragment childFragment : childrenFragments) { recursiveCteNode.addChild(childFragment.getPlanRoot()); } List> materializedResultExprLists = Lists.newArrayList(); - for (int i = 0; i < resultExpressionLists.size(); ++i) { - List resultExpressionList = resultExpressionLists.get(i); + for (int i = 0; i < recursiveCte.getRegularChildrenOutputs().size(); ++i) { + List resultExpressionList = recursiveCte.getRegularChildrenOutputs().get(i); List exprList = Lists.newArrayList(); - Preconditions.checkState(resultExpressionList.size() == outputSlotDescs.size()); for (int j = 0; j < resultExpressionList.size(); ++j) { exprList.add(ExpressionTranslator.translate(resultExpressionList.get(j), context)); - // TODO: reconsider this, we may change nullable info in previous nereids rules not here. - outputSlotDescs.get(j) - .setIsNullable(outputSlotDescs.get(j).getIsNullable() || exprList.get(j).isNullable()); } materializedResultExprLists.add(exprList); } recursiveCteNode.setMaterializedResultExprLists(materializedResultExprLists); - Preconditions.checkState(recursiveCteNode.getMaterializedResultExprLists().size() - == recursiveCteNode.getChildren().size()); - PlanFragment recursiveCteFragment; - if (childrenFragments.isEmpty()) { - recursiveCteFragment = createPlanFragment(recursiveCteNode, - DataPartition.UNPARTITIONED, recursiveCte); - context.addPlanFragment(recursiveCteFragment); - } else { - int childrenSize = childrenFragments.size(); - recursiveCteFragment = childrenFragments.get(childrenSize - 1); - for (int i = childrenSize - 2; i >= 0; i--) { - context.mergePlanFragment(childrenFragments.get(i), recursiveCteFragment); - for (PlanFragment child : childrenFragments.get(i).getChildren()) { - recursiveCteFragment.addChild(child); - } + int childrenSize = childrenFragments.size(); + PlanFragment recursiveCteFragment = childrenFragments.get(childrenSize - 1); + for (int i = childrenSize - 2; i >= 0; i--) { + context.mergePlanFragment(childrenFragments.get(i), recursiveCteFragment); + for (PlanFragment child : childrenFragments.get(i).getChildren()) { + recursiveCteFragment.addChild(child); } - setPlanRoot(recursiveCteFragment, recursiveCteNode, recursiveCte); } + setPlanRoot(recursiveCteFragment, recursiveCteNode, recursiveCte); recursiveCteFragment.updateDataPartition(DataPartition.UNPARTITIONED); recursiveCteFragment.setOutputPartition(DataPartition.UNPARTITIONED); @@ -2274,10 +2249,17 @@ public PlanFragment visitPhysicalRecursiveCte(PhysicalRecursiveCte recursiveCte, } @Override - public PlanFragment visitPhysicalRecursiveCteRecursiveChild( - PhysicalRecursiveCteRecursiveChild recursiveChild, + public PlanFragment visitPhysicalRecursiveUnionAnchor( + PhysicalRecursiveUnionAnchor recursiveUnionAnchor, PlanTranslatorContext context) { - return recursiveChild.child().accept(this, context); + return recursiveUnionAnchor.child().accept(this, context); + } + + @Override + public PlanFragment visitPhysicalRecursiveUnionProducer( + PhysicalRecursiveUnionProducer recursiveUnionProducer, + PlanTranslatorContext context) { + return recursiveUnionProducer.child().accept(this, context); } /** @@ -2299,7 +2281,7 @@ public PlanFragment visitPhysicalSetOperation( for (Plan plan : setOperation.children()) { childrenFragments.add(plan.accept(this, context)); } - + List> distributeExprLists = getDistributeExprs(setOperation.children().toArray(new Plan[0])); TupleDescriptor setTuple = generateTupleDesc(setOperation.getOutput(), null, context); SetOperationNode setOperationNode; @@ -2313,7 +2295,6 @@ public PlanFragment visitPhysicalSetOperation( } else { throw new RuntimeException("not support set operation type " + setOperation); } - List> distributeExprLists = getDistributeExprs(setOperation.children().toArray(new Plan[0])); setOperationNode.setChildrenDistributeExprLists(distributeExprLists); setOperationNode.setNereidsId(setOperation.getId()); context.getNereidsIdToPlanNodeIdMap().put(setOperation.getId(), setOperationNode.getId()); 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 35d64c4455e2e8..4133ec25337235 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 @@ -1127,6 +1127,8 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor { private final Map selectHintMap; + // recursive cte is in form of union[all], and in visitSetOperation method, we try to reduceToLogicalPlanTree + // for UNION. We should not do it for recursive cte, so this flag is to indicate if we meet recursive cte private boolean isInRecursiveCteContext = false; public LogicalPlanBuilder(Map selectHintMap) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/MemoPatterns.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/MemoPatterns.java index ec53deafaadbd3..fa9d191f5e2af3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/MemoPatterns.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/pattern/MemoPatterns.java @@ -28,7 +28,6 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalIntersect; import org.apache.doris.nereids.trees.plans.logical.LogicalLeaf; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCte; import org.apache.doris.nereids.trees.plans.logical.LogicalRelation; import org.apache.doris.nereids.trees.plans.logical.LogicalSetOperation; import org.apache.doris.nereids.trees.plans.logical.LogicalUnary; @@ -206,29 +205,6 @@ default PatternDescriptor logicalUnion() { defaultPromise()); } - /** - * create a LogicalRecursiveCte pattern. - */ - default PatternDescriptor - logicalRecursiveCte( - PatternDescriptor... children) { - return new PatternDescriptor( - new TypePattern(LogicalRecursiveCte.class, - Arrays.stream(children) - .map(PatternDescriptor::getPattern) - .toArray(Pattern[]::new)), - defaultPromise()); - } - - /** - * create a logicalRecursiveCte group. - */ - default PatternDescriptor logicalRecursiveCte() { - return new PatternDescriptor( - new TypePattern(LogicalRecursiveCte.class, multiGroup().pattern), - defaultPromise()); - } - /** * create a logicalExcept pattern. */ 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 d7e4e5b67cb34b..594c35966fbdc2 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 @@ -33,7 +33,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalIntersect; 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.PhysicalRecursiveCte; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnion; import org.apache.doris.nereids.trees.plans.physical.PhysicalRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalSetOperation; import org.apache.doris.nereids.trees.plans.physical.PhysicalTopN; @@ -78,11 +78,12 @@ public Plan visit(Plan plan, CascadesContext context) { } @Override - public PhysicalRecursiveCte visitPhysicalRecursiveCte(PhysicalRecursiveCte recursiveCte, CascadesContext context) { - for (Plan child : recursiveCte.children()) { + public PhysicalRecursiveUnion visitPhysicalRecursiveUnion( + PhysicalRecursiveUnion recursiveUnion, CascadesContext context) { + for (Plan child : recursiveUnion.children()) { child.accept(this, context); } - return recursiveCte; + return recursiveUnion; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java index 6af9bc5e0917f0..6ced632262280a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/processor/pre/PullUpSubqueryAliasToCTE.java @@ -83,7 +83,7 @@ public Plan visitLogicalCTE(LogicalCTE logicalCTE, StatementCont subQueryAliasesOfCte.addAll(logicalCTE.getAliasQueries()); subQueryAliasesOfCte.addAll(aliasQueries); aliasQueries = new ArrayList<>(); - return new LogicalCTE<>(newLogicalCTE.isRecursiveCte(), subQueryAliasesOfCte, + return new LogicalCTE<>(newLogicalCTE.isRecursive(), subQueryAliasesOfCte, (LogicalPlan) newLogicalCTE.child()); } return cte; @@ -98,7 +98,7 @@ private Plan createCteForRootNode(Plan plan, StatementContext context) { subQueryAliases.addAll(logicalCTE.getAliasQueries()); subQueryAliases.addAll(aliasQueries); return topPlan.withChildren( - new LogicalCTE<>(logicalCTE.isRecursiveCte(), subQueryAliases, + new LogicalCTE<>(logicalCTE.isRecursive(), subQueryAliases, (LogicalPlan) topPlan.child(0))); } return topPlan.withChildren( 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 a7dfc11046f8aa..3c49c9fbec500c 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 @@ -53,15 +53,15 @@ 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.PhysicalRecursiveCte; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteRecursiveChild; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnion; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionProducer; 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; +import org.apache.doris.nereids.trees.plans.physical.PhysicalWorkTableReference; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.types.DataType; import org.apache.doris.nereids.util.JoinUtils; @@ -154,7 +154,7 @@ public PhysicalProperties visitPhysicalFileScan(PhysicalFileScan fileScan, PlanC } @Override - public PhysicalProperties visitPhysicalRecursiveCteScan(PhysicalRecursiveCteScan cteScan, PlanContext context) { + public PhysicalProperties visitPhysicalWorkTableReference(PhysicalWorkTableReference cteScan, PlanContext context) { return PhysicalProperties.ANY; } @@ -545,13 +545,14 @@ public PhysicalProperties visitPhysicalSetOperation(PhysicalSetOperation setOper } @Override - public PhysicalProperties visitPhysicalRecursiveCte(PhysicalRecursiveCte recursiveCte, PlanContext context) { + public PhysicalProperties visitPhysicalRecursiveUnion( + PhysicalRecursiveUnion recursiveCte, PlanContext context) { return PhysicalProperties.GATHER; } @Override - public PhysicalProperties visitPhysicalRecursiveCteRecursiveChild( - PhysicalRecursiveCteRecursiveChild recursiveChild, + public PhysicalProperties visitPhysicalRecursiveUnionProducer( + PhysicalRecursiveUnionProducer recursiveChild, PlanContext context) { return PhysicalProperties.MUST_SHUFFLE; } 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 a5bf8e2fe9268d..785b912d2d6be3 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 @@ -54,7 +54,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink; import org.apache.doris.nereids.trees.plans.physical.PhysicalPartitionTopN; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCte; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnion; 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; @@ -320,12 +320,9 @@ public Void visitPhysicalUnion(PhysicalUnion union, PlanContext context) { } @Override - public Void visitPhysicalRecursiveCte(PhysicalRecursiveCte recursiveCte, PlanContext context) { - List requestGather = Lists.newArrayListWithCapacity(context.arity()); - for (int i = context.arity(); i > 0; --i) { - requestGather.add(PhysicalProperties.GATHER); - } - addRequestPropertyToChildren(requestGather); + public Void visitPhysicalRecursiveUnion(PhysicalRecursiveUnion recursiveUnion, + PlanContext context) { + addRequestPropertyToChildren(PhysicalProperties.GATHER, PhysicalProperties.GATHER); return null; } 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 2790553de579e3..d8c119103a8a71 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 @@ -89,9 +89,9 @@ import org.apache.doris.nereids.rules.implementation.LogicalOneRowRelationToPhysicalOneRowRelation; import org.apache.doris.nereids.rules.implementation.LogicalPartitionTopNToPhysicalPartitionTopN; import org.apache.doris.nereids.rules.implementation.LogicalProjectToPhysicalProject; -import org.apache.doris.nereids.rules.implementation.LogicalRecursiveCteRecursiveChildToPhysicalRecursiveCteRecursiveChild; -import org.apache.doris.nereids.rules.implementation.LogicalRecursiveCteScanToPhysicalRecursiveCteScan; -import org.apache.doris.nereids.rules.implementation.LogicalRecursiveCteToPhysicalRecursiveCte; +import org.apache.doris.nereids.rules.implementation.LogicalRecursiveUnionAnchorToPhysicalRecursiveUnionAnchor; +import org.apache.doris.nereids.rules.implementation.LogicalRecursiveUnionProducerToPhysicalRecursiveUnionProducer; +import org.apache.doris.nereids.rules.implementation.LogicalRecursiveUnionToPhysicalRecursiveUnion; import org.apache.doris.nereids.rules.implementation.LogicalRepeatToPhysicalRepeat; import org.apache.doris.nereids.rules.implementation.LogicalResultSinkToPhysicalResultSink; import org.apache.doris.nereids.rules.implementation.LogicalSchemaScanToPhysicalSchemaScan; @@ -100,6 +100,7 @@ import org.apache.doris.nereids.rules.implementation.LogicalTopNToPhysicalTopN; import org.apache.doris.nereids.rules.implementation.LogicalUnionToPhysicalUnion; import org.apache.doris.nereids.rules.implementation.LogicalWindowToPhysicalWindow; +import org.apache.doris.nereids.rules.implementation.LogicalWorkTableReferenceToPhysicalWorkTableReference; import org.apache.doris.nereids.rules.implementation.SplitAggMultiPhase; import org.apache.doris.nereids.rules.implementation.SplitAggMultiPhaseWithoutGbyKey; import org.apache.doris.nereids.rules.implementation.SplitAggWithoutDistinct; @@ -208,7 +209,7 @@ public class RuleSet { .add(new LogicalJdbcScanToPhysicalJdbcScan()) .add(new LogicalOdbcScanToPhysicalOdbcScan()) .add(new LogicalEsScanToPhysicalEsScan()) - .add(new LogicalRecursiveCteScanToPhysicalRecursiveCteScan()) + .add(new LogicalWorkTableReferenceToPhysicalWorkTableReference()) .add(new LogicalProjectToPhysicalProject()) .add(new LogicalLimitToPhysicalLimit()) .add(new LogicalWindowToPhysicalWindow()) @@ -224,8 +225,9 @@ public class RuleSet { .add(SplitAggWithoutDistinct.INSTANCE) .add(SplitAggMultiPhase.INSTANCE) .add(SplitAggMultiPhaseWithoutGbyKey.INSTANCE) - .add(new LogicalRecursiveCteToPhysicalRecursiveCte()) - .add(new LogicalRecursiveCteRecursiveChildToPhysicalRecursiveCteRecursiveChild()) + .add(new LogicalRecursiveUnionToPhysicalRecursiveUnion()) + .add(new LogicalRecursiveUnionProducerToPhysicalRecursiveUnionProducer()) + .add(new LogicalRecursiveUnionAnchorToPhysicalRecursiveUnionAnchor()) .add(new LogicalUnionToPhysicalUnion()) .add(new LogicalExceptToPhysicalExcept()) .add(new LogicalIntersectToPhysicalIntersect()) 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 21f1ae4bdbf46b..2587481256b798 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 @@ -546,7 +546,7 @@ public enum RuleType { LOGICAL_JDBC_SCAN_TO_PHYSICAL_JDBC_SCAN_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_ODBC_SCAN_TO_PHYSICAL_ODBC_SCAN_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_ES_SCAN_TO_PHYSICAL_ES_SCAN_RULE(RuleTypeClass.IMPLEMENTATION), - LOGICAL_RECURSIVE_CTE_SCAN_TO_PHYSICAL_RECUSIVE_CTE_SCAN_RULE(RuleTypeClass.IMPLEMENTATION), + LOGICAL_WORK_TABLE_REFERENCE_TO_PHYSICAL_WORK_TABLE_REFERENCE(RuleTypeClass.IMPLEMENTATION), LOGICAL_BLACKHOLE_SINK_TO_PHYSICAL_BLACKHOLE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_OLAP_TABLE_SINK_TO_PHYSICAL_OLAP_TABLE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), LOGICAL_HIVE_TABLE_SINK_TO_PHYSICAL_HIVE_TABLE_SINK_RULE(RuleTypeClass.IMPLEMENTATION), @@ -568,8 +568,9 @@ public enum RuleType { COUNT_ON_INDEX_WITHOUT_PROJECT(RuleTypeClass.IMPLEMENTATION), TWO_PHASE_AGGREGATE_WITH_DISTINCT(RuleTypeClass.IMPLEMENTATION), LOGICAL_UNION_TO_PHYSICAL_UNION(RuleTypeClass.IMPLEMENTATION), - LOGICAL_RECURSIVE_CTE_TO_PHYSICAL_RECURSIVE_CTE(RuleTypeClass.IMPLEMENTATION), - LOGICAL_RECURSIVE_CTE_RECURSIVE_CHILD_TO_PHYSICAL_RECURSIVE_CTE_RECURSIVE_CHILD(RuleTypeClass.IMPLEMENTATION), + LOGICAL_RECURSIVE_UNION_TO_PHYSICAL_RECURSIVE_UNION(RuleTypeClass.IMPLEMENTATION), + LOGICAL_RECURSIVE_UNION_PRODUCER_TO_PHYSICAL_RECURSIVE_UNION_PRODUCER(RuleTypeClass.IMPLEMENTATION), + LOGICAL_RECURSIVE_UNION_ANCHOR_TO_PHYSICAL_RECURSIVE_UNION_ANCHOR(RuleTypeClass.IMPLEMENTATION), LOGICAL_EXCEPT_TO_PHYSICAL_EXCEPT(RuleTypeClass.IMPLEMENTATION), LOGICAL_INTERSECT_TO_PHYSICAL_INTERSECT(RuleTypeClass.IMPLEMENTATION), LOGICAL_GENERATE_TO_PHYSICAL_GENERATE(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 index 0c65c8af560001..f59d88a8b84766 100644 --- 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 @@ -31,18 +31,17 @@ import org.apache.doris.nereids.trees.expressions.StatementScopeIdGenerator; import org.apache.doris.nereids.trees.expressions.functions.scalar.Nullable; import org.apache.doris.nereids.trees.plans.Plan; -import org.apache.doris.nereids.trees.plans.algebra.SetOperation; 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.LogicalProject; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCte; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCteRecursiveChild; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCteScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnion; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnionAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnionProducer; import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; import org.apache.doris.nereids.trees.plans.logical.LogicalUnion; -import org.apache.doris.nereids.trees.plans.logical.ProjectProcessor; +import org.apache.doris.nereids.trees.plans.logical.LogicalWorkTableReference; import org.apache.doris.nereids.types.DataType; import com.google.common.base.Preconditions; @@ -52,7 +51,6 @@ import java.util.ArrayList; import java.util.HashSet; import java.util.List; -import java.util.Optional; import java.util.Set; import java.util.stream.Collectors; @@ -68,6 +66,10 @@ public class AnalyzeCTE extends OneAnalysisRuleFactory { public Rule build() { return logicalCTE().thenApply(ctx -> { LogicalCTE logicalCTE = ctx.root; + if (logicalCTE.isRecursive() + && !ctx.connectContext.getSessionVariable().isEnableNereidsDistributePlanner()) { + throw new AnalysisException("please set enable_nereids_distribute_planner=true to use RECURSIVE CTE"); + } // step 0. check duplicate cte name Set uniqueAlias = Sets.newHashSet(); @@ -84,7 +86,7 @@ public Rule build() { // step 1. analyzed all cte plan Pair>> result = analyzeCte(logicalCTE, ctx.cascadesContext); CascadesContext outerCascadesCtx = CascadesContext.newContextWithCteContext( - ctx.cascadesContext, logicalCTE.child(), result.first, Optional.empty(), ImmutableList.of()); + ctx.cascadesContext, logicalCTE.child(), result.first, null); outerCascadesCtx.withPlanProcess(ctx.cascadesContext.showPlanProcess(), () -> { outerCascadesCtx.newAnalyzer().analyze(); }); @@ -109,7 +111,9 @@ private Pair>> analyzeCte( List> cteProducerPlans = new ArrayList<>(); for (LogicalSubQueryAlias aliasQuery : aliasQueries) { // we should use a chain to ensure visible of cte - if (aliasQuery.isRecursiveCte() && logicalCTE.isRecursiveCte()) { + if (logicalCTE.isRecursive() && aliasQuery.isRecursiveCte()) { + // if we have WITH RECURSIVE keyword, logicalCTE.isRecursive() will be true, + // but we still need to check if aliasQuery is a real recursive cte or just normal cte Pair> result = analyzeRecursiveCte(aliasQuery, outerCteCtx, cascadesContext); outerCteCtx = result.first; @@ -117,7 +121,7 @@ private Pair>> analyzeCte( } else { LogicalPlan parsedCtePlan = (LogicalPlan) aliasQuery.child(); CascadesContext innerCascadesCtx = CascadesContext.newContextWithCteContext( - cascadesContext, parsedCtePlan, outerCteCtx, Optional.empty(), ImmutableList.of()); + cascadesContext, parsedCtePlan, outerCteCtx, null); innerCascadesCtx.withPlanProcess(cascadesContext.showPlanProcess(), () -> { innerCascadesCtx.newAnalyzer().analyze(); }); @@ -139,44 +143,56 @@ private Pair> analyzeRecursiveCte(LogicalSu CTEContext outerCteCtx, CascadesContext cascadesContext) { Preconditions.checkArgument(aliasQuery.isRecursiveCte(), "alias query must be recursive cte"); LogicalPlan parsedCtePlan = (LogicalPlan) aliasQuery.child(); - if (!(parsedCtePlan instanceof LogicalUnion) || parsedCtePlan.children().size() != 2) { + if (!(parsedCtePlan instanceof LogicalUnion)) { throw new AnalysisException(String.format("recursive cte must be union, don't support %s", parsedCtePlan.getClass().getSimpleName())); } - // analyze anchor child, its output list will be recursive cte temp table's schema + + if (parsedCtePlan.arity() != 2) { + throw new AnalysisException(String.format("recursive cte must have 2 children, but it has %d", + parsedCtePlan.arity())); + } + // analyze anchor child, its output list will be LogicalWorkTableReference's schema + // also pass cte name to anchor side to let the relation binding happy. Then we check if the anchor reference + // the recursive cte itself and report a user-friendly error message like bellow LogicalPlan anchorChild = (LogicalPlan) parsedCtePlan.child(0); + CTEContext recursiveCteCtx = new CTEContext(StatementScopeIdGenerator.newCTEId(), aliasQuery.getAlias(), null); CascadesContext innerAnchorCascadesCtx = CascadesContext.newContextWithCteContext( - cascadesContext, anchorChild, outerCteCtx, Optional.of(aliasQuery.getAlias()), ImmutableList.of()); + cascadesContext, anchorChild, outerCteCtx, recursiveCteCtx); innerAnchorCascadesCtx.withPlanProcess(cascadesContext.showPlanProcess(), () -> { innerAnchorCascadesCtx.newAnalyzer().analyze(); }); cascadesContext.addPlanProcesses(innerAnchorCascadesCtx.getPlanProcesses()); LogicalPlan analyzedAnchorChild = (LogicalPlan) innerAnchorCascadesCtx.getRewritePlan(); - Set recursiveCteScans = analyzedAnchorChild - .collect(LogicalRecursiveCteScan.class::isInstance); - for (LogicalRecursiveCteScan cteScan : recursiveCteScans) { - if (cteScan.getTable().getName().equalsIgnoreCase(aliasQuery.getAlias())) { + Set recursiveCteScans = analyzedAnchorChild + .collect(LogicalWorkTableReference.class::isInstance); + for (LogicalWorkTableReference cteScan : recursiveCteScans) { + if (cteScan.getTableName().equalsIgnoreCase(aliasQuery.getAlias())) { throw new AnalysisException( String.format("recursive reference to query %s must not appear within its non-recursive term", aliasQuery.getAlias())); } } checkColumnAlias(aliasQuery, analyzedAnchorChild.getOutput()); - // make all output nullable + // make all output nullable, the behavior is same as pg. It's much simpler than complex derivation of nullable + // also we change the output same as cte's column aliases if it has. Because recursive part will try to use + // column aliases first then the original output slot. analyzedAnchorChild = forceOutputNullable(analyzedAnchorChild, aliasQuery.getColumnAliases().orElse(ImmutableList.of())); - // analyze recursive child + analyzedAnchorChild = new LogicalRecursiveUnionAnchor<>(recursiveCteCtx.getCteId(), analyzedAnchorChild); + // analyze recursive child, analyzedAnchorChild.getOutput() will be LogicalWorkTableReference's schema LogicalPlan recursiveChild = (LogicalPlan) parsedCtePlan.child(1); + recursiveCteCtx.setRecursiveCteOutputs(analyzedAnchorChild.getOutput()); CascadesContext innerRecursiveCascadesCtx = CascadesContext.newContextWithCteContext( - cascadesContext, recursiveChild, outerCteCtx, Optional.of(aliasQuery.getAlias()), - analyzedAnchorChild.getOutput()); + cascadesContext, recursiveChild, outerCteCtx, recursiveCteCtx); innerRecursiveCascadesCtx.withPlanProcess(cascadesContext.showPlanProcess(), () -> { innerRecursiveCascadesCtx.newAnalyzer().analyze(); }); cascadesContext.addPlanProcesses(innerRecursiveCascadesCtx.getPlanProcesses()); LogicalPlan analyzedRecursiveChild = (LogicalPlan) innerRecursiveCascadesCtx.getRewritePlan(); - List recursiveCteScanList = analyzedRecursiveChild - .collectToList(LogicalRecursiveCteScan.class::isInstance); + List recursiveCteScanList = analyzedRecursiveChild + .collectToList(item -> item instanceof LogicalWorkTableReference + && ((LogicalWorkTableReference) item).getCteId().equals(recursiveCteCtx.getCteId())); if (recursiveCteScanList.size() > 1) { throw new AnalysisException(String.format("recursive reference to query %s must not appear more than once", aliasQuery.getAlias())); @@ -187,43 +203,34 @@ private Pair> analyzeRecursiveCte(LogicalSu anchorChildOutputTypes.add(slot.getDataType()); } List recursiveChildOutputs = analyzedRecursiveChild.getOutput(); + // anchor and recursive are union's children, so their output size must be same + Preconditions.checkState(anchorChildOutputs.size() == recursiveChildOutputs.size(), + "anchor and recursive child's output size must be same"); for (int i = 0; i < recursiveChildOutputs.size(); ++i) { - if (!recursiveChildOutputs.get(i).getDataType().equals(anchorChildOutputTypes.get(i))) { + if (!recursiveChildOutputs.get(i).getDataType().equalsForRecursiveCte(anchorChildOutputTypes.get(i))) { throw new AnalysisException(String.format("%s recursive child's %d column's datatype in select list %s " - + "is different from anchor child's output datatype %s, please add cast manually " - + "to get expect datatype", aliasQuery.getAlias(), i + 1, + + "is different from anchor child's output datatype %s, please add cast manually " + + "to get expect datatype", aliasQuery.getAlias(), i + 1, recursiveChildOutputs.get(i).getDataType(), anchorChildOutputTypes.get(i))); } } - analyzedRecursiveChild = new LogicalRecursiveCteRecursiveChild<>(aliasQuery.getAlias(), + // make analyzedRecursiveChild's outputs all nullable and keep slot name unchanged + analyzedRecursiveChild = new LogicalRecursiveUnionProducer<>(aliasQuery.getAlias(), forceOutputNullable(analyzedRecursiveChild, ImmutableList.of())); - // create LogicalRecursiveCte + // create LogicalRecursiveUnion LogicalUnion logicalUnion = (LogicalUnion) parsedCtePlan; - LogicalRecursiveCte analyzedCtePlan = new LogicalRecursiveCte(aliasQuery.getAlias(), - logicalUnion.getQualifier() == SetOperation.Qualifier.ALL, - ImmutableList.of(analyzedAnchorChild, analyzedRecursiveChild)); - List> childrenProjections = analyzedCtePlan.collectChildrenProjections(); - int childrenProjectionSize = childrenProjections.size(); - ImmutableList.Builder> childrenOutputs = ImmutableList - .builderWithExpectedSize(childrenProjectionSize); - ImmutableList.Builder newChildren = ImmutableList.builderWithExpectedSize(childrenProjectionSize); - for (int i = 0; i < childrenProjectionSize; i++) { - Plan newChild; - Plan child = analyzedCtePlan.child(i); - if (childrenProjections.get(i).stream().allMatch(SlotReference.class::isInstance)) { - newChild = child; - } else { - List parentProject = childrenProjections.get(i); - newChild = ProjectProcessor.tryProcessProject(parentProject, child) - .orElseGet(() -> new LogicalProject<>(parentProject, child)); - } - newChildren.add(newChild); - childrenOutputs.add((List) (List) newChild.getOutput()); + ImmutableList.Builder newOutputs = ImmutableList + .builderWithExpectedSize(anchorChildOutputs.size()); + for (Slot slot : anchorChildOutputs) { + newOutputs.add(new SlotReference(slot.toSql(), slot.getDataType(), slot.nullable(), ImmutableList.of())); } - analyzedCtePlan = analyzedCtePlan.withChildrenAndTheirOutputs(newChildren.build(), childrenOutputs.build()); - List newOutputs = analyzedCtePlan.buildNewOutputs(); - analyzedCtePlan = analyzedCtePlan.withNewOutputs(newOutputs); + LogicalRecursiveUnion analyzedCtePlan = new LogicalRecursiveUnion( + aliasQuery.getAlias(), + logicalUnion.getQualifier(), + newOutputs.build(), + ImmutableList.of(analyzedAnchorChild.getOutput(), analyzedRecursiveChild.getOutput()), + ImmutableList.of(analyzedAnchorChild, analyzedRecursiveChild)); CTEId cteId = StatementScopeIdGenerator.newCTEId(); LogicalSubQueryAlias logicalSubQueryAlias = aliasQuery.withChildren(ImmutableList.of(analyzedCtePlan)); 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 44dfbc563a71db..237cf926c7b556 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 @@ -27,7 +27,6 @@ import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.OlapTable; import org.apache.doris.catalog.Partition; -import org.apache.doris.catalog.RecursiveCteTempTable; import org.apache.doris.catalog.SchemaTable; import org.apache.doris.catalog.SchemaTable.SchemaColumn; import org.apache.doris.catalog.TableIf; @@ -91,12 +90,12 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOdbcScan; import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan; import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCteScan; import org.apache.doris.nereids.trees.plans.logical.LogicalSchemaScan; 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.LogicalTestScan; import org.apache.doris.nereids.trees.plans.logical.LogicalView; +import org.apache.doris.nereids.trees.plans.logical.LogicalWorkTableReference; import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.AutoCloseSessionVariable; @@ -170,37 +169,39 @@ private LogicalPlan bindWithCurrentDb(CascadesContext cascadesContext, UnboundRe leading.putRelationIdAndTableName(Pair.of(consumer.getRelationId(), tableName)); leading.getRelationIdToScanMap().put(consumer.getRelationId(), consumer); } + if (cascadesContext.getRecursiveCteContext().isPresent()) { + // we are analyzing recursive CTE's recursive child, must inline all used CTEs + cascadesContext.getStatementContext().addToMustLineCTEs(cteContext.getCteId()); + } return consumer; } } - LogicalPlan scan; - if (tableName.equalsIgnoreCase(cascadesContext.getCurrentRecursiveCteName().orElse(""))) { + LogicalPlan plan; + // check if it is a recursive CTE's name + if (cascadesContext.getRecursiveCteContext().isPresent() + && cascadesContext.getRecursiveCteContext().get().findCTEContext(tableName).isPresent()) { if (cascadesContext.isAnalyzingRecursiveCteAnchorChild()) { throw new AnalysisException( String.format("recursive reference to query %s must not appear within its non-recursive term", - cascadesContext.getCurrentRecursiveCteName().get())); - } - ImmutableList.Builder schema = new ImmutableList.Builder<>(); - for (Slot slot : cascadesContext.getRecursiveCteOutputs()) { - schema.add(new Column(slot.getName(), slot.getDataType().toCatalogDataType(), slot.nullable())); + tableName)); } - RecursiveCteTempTable cteTempTable = new RecursiveCteTempTable(tableName, schema.build()); - scan = new LogicalRecursiveCteScan(cascadesContext.getStatementContext().getNextRelationId(), - cteTempTable, unboundRelation.getNameParts()); + plan = new LogicalWorkTableReference(cascadesContext.getStatementContext().getNextRelationId(), + cascadesContext.getRecursiveCteContext().get().getCteId(), + cascadesContext.getRecursiveCteOutputs(), unboundRelation.getNameParts()); } else { List tableQualifier = RelationUtil.getQualifierName( cascadesContext.getConnectContext(), unboundRelation.getNameParts()); TableIf table = cascadesContext.getStatementContext().getAndCacheTable(tableQualifier, TableFrom.QUERY, Optional.of(unboundRelation)); - scan = getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); + plan = getLogicalPlan(table, unboundRelation, tableQualifier, cascadesContext); if (cascadesContext.isLeadingJoin()) { LeadingHint leading = (LeadingHint) cascadesContext.getHintMap().get("Leading"); leading.putRelationIdAndTableName(Pair.of(unboundRelation.getRelationId(), tableName)); - leading.getRelationIdToScanMap().put(unboundRelation.getRelationId(), scan); + leading.getRelationIdToScanMap().put(unboundRelation.getRelationId(), plan); } } - return scan; + return plan; } private LogicalPlan bind(CascadesContext cascadesContext, UnboundRelation unboundRelation) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java index c90abf0b02bacd..6d0b20801676aa 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/CollectRelation.java @@ -107,10 +107,9 @@ private CTEContext collectFromCte( for (LogicalSubQueryAlias aliasQuery : aliasQueries) { // we should use a chain to ensure visible of cte LogicalPlan parsedCtePlan = (LogicalPlan) aliasQuery.child(); - // 看起来需要在CascadesContext中添加当前CTE的name,以便判断自引用 CascadesContext innerCascadesCtx = CascadesContext.newContextWithCteContext( cascadesContext, parsedCtePlan, outerCteCtx, aliasQuery.isRecursiveCte() - ? Optional.of(aliasQuery.getAlias()) : Optional.empty(), ImmutableList.of()); + ? new CTEContext(CTEId.DEFAULT, aliasQuery.getAlias(), null) : null); innerCascadesCtx.newTableCollector(true).collect(); LogicalPlan analyzedCtePlan = (LogicalPlan) innerCascadesCtx.getRewritePlan(); // cteId is not used in CollectTable stage @@ -129,8 +128,7 @@ private Plan collectFromAny(MatchingContext ctx) { if (e instanceof SubqueryExpr) { SubqueryExpr subqueryExpr = (SubqueryExpr) e; CascadesContext subqueryContext = CascadesContext.newContextWithCteContext( - ctx.cascadesContext, subqueryExpr.getQueryPlan(), ctx.cteContext, Optional.empty(), - ImmutableList.of()); + ctx.cascadesContext, subqueryExpr.getQueryPlan(), ctx.cteContext, null); subqueryContext.keepOrShowPlanProcess(ctx.cascadesContext.showPlanProcess(), () -> subqueryContext.newTableCollector(true).collect()); ctx.cascadesContext.addPlanProcesses(subqueryContext.getPlanProcesses()); @@ -182,10 +180,12 @@ private void collectFromUnboundRelation(CascadesContext cascadesContext, List nameParts, TableFrom tableFrom, Optional unboundRelation) { if (nameParts.size() == 1) { String tableName = nameParts.get(0); - if (cascadesContext.getCurrentRecursiveCteName().isPresent() - && tableName.equalsIgnoreCase(cascadesContext.getCurrentRecursiveCteName().get())) { + // check if it is a recursive CTE's name + if (cascadesContext.getRecursiveCteContext().isPresent() + && cascadesContext.getRecursiveCteContext().get().findCTEContext(tableName).isPresent()) { return; } + // check if it is a CTE's name CTEContext cteContext = cascadesContext.getCteContext().findCTEContext(tableName).orElse(null); if (cteContext != null) { 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 05a35dc6ed8460..3db749ab5ed785 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 @@ -52,7 +52,6 @@ import java.util.HashSet; import java.util.List; import java.util.Objects; -import java.util.Optional; import java.util.Set; /** @@ -228,8 +227,7 @@ private AnalyzedResult analyzeSubquery(SubqueryExpr expr) { throw new IllegalStateException("Missing CascadesContext"); } CascadesContext subqueryContext = CascadesContext.newContextWithCteContext( - cascadesContext, expr.getQueryPlan(), cascadesContext.getCteContext(), Optional.empty(), - ImmutableList.of()); + cascadesContext, expr.getQueryPlan(), cascadesContext.getCteContext(), null); // don't use `getScope()` because we only need `getScope().getOuterScope()` and `getScope().getSlots()` // otherwise unexpected errors may occur Scope subqueryScope = new Scope(getScope().getOuterScope(), diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteToPhysicalRecursiveCte.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionAnchorToPhysicalRecursiveUnionAnchor.java similarity index 62% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteToPhysicalRecursiveCte.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionAnchorToPhysicalRecursiveUnionAnchor.java index 12ac89df19c644..13057ab789457b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteToPhysicalRecursiveCte.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionAnchorToPhysicalRecursiveUnionAnchor.java @@ -19,21 +19,19 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCte; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionAnchor; /** - * Implementation rule that convert logical Recursive Cte to Physical Recursive Cte. + * Implementation rule that convert LogicalRecursiveUnionAnchor to PhysicalRecursiveUnionAnchor. */ -public class LogicalRecursiveCteToPhysicalRecursiveCte extends OneImplementationRuleFactory { +public class LogicalRecursiveUnionAnchorToPhysicalRecursiveUnionAnchor + extends OneImplementationRuleFactory { @Override public Rule build() { - return logicalRecursiveCte().then(recursiveCte -> - new PhysicalRecursiveCte(recursiveCte.getCteName(), - recursiveCte.isUnionAll(), - recursiveCte.getOutputs(), - recursiveCte.getRegularChildrenOutputs(), - recursiveCte.getLogicalProperties(), - recursiveCte.children()) - ).toRule(RuleType.LOGICAL_RECURSIVE_CTE_TO_PHYSICAL_RECURSIVE_CTE); + return logicalRecursiveUnionAnchor().then(recursiveCte -> new PhysicalRecursiveUnionAnchor( + recursiveCte.getCteId(), + recursiveCte.getLogicalProperties(), + recursiveCte.child())) + .toRule(RuleType.LOGICAL_RECURSIVE_UNION_ANCHOR_TO_PHYSICAL_RECURSIVE_UNION_ANCHOR); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteRecursiveChildToPhysicalRecursiveCteRecursiveChild.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionProducerToPhysicalRecursiveUnionProducer.java similarity index 74% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteRecursiveChildToPhysicalRecursiveCteRecursiveChild.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionProducerToPhysicalRecursiveUnionProducer.java index 689e550fa3a113..55d2ec91d8fa44 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteRecursiveChildToPhysicalRecursiveCteRecursiveChild.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionProducerToPhysicalRecursiveUnionProducer.java @@ -19,19 +19,19 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteRecursiveChild; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionProducer; /** - * Implementation rule that convert logical recursive cte's recursive child to physical recursive child. + * Implementation rule that convert LogicalRecursiveUnionProducer to PhysicalRecursiveUnionProducer. */ -public class LogicalRecursiveCteRecursiveChildToPhysicalRecursiveCteRecursiveChild +public class LogicalRecursiveUnionProducerToPhysicalRecursiveUnionProducer extends OneImplementationRuleFactory { @Override public Rule build() { - return logicalRecursiveCteRecursiveChild().then(recursiveCte -> new PhysicalRecursiveCteRecursiveChild( + return logicalRecursiveUnionProducer().then(recursiveCte -> new PhysicalRecursiveUnionProducer( recursiveCte.getCteName(), recursiveCte.getLogicalProperties(), recursiveCte.child())) - .toRule(RuleType.LOGICAL_RECURSIVE_CTE_RECURSIVE_CHILD_TO_PHYSICAL_RECURSIVE_CTE_RECURSIVE_CHILD); + .toRule(RuleType.LOGICAL_RECURSIVE_UNION_PRODUCER_TO_PHYSICAL_RECURSIVE_UNION_PRODUCER); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteScanToPhysicalRecursiveCteScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionToPhysicalRecursiveUnion.java similarity index 58% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteScanToPhysicalRecursiveCteScan.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionToPhysicalRecursiveUnion.java index 8714c280bb9064..d45cdba33a8810 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveCteScanToPhysicalRecursiveCteScan.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalRecursiveUnionToPhysicalRecursiveUnion.java @@ -19,24 +19,20 @@ import org.apache.doris.nereids.rules.Rule; import org.apache.doris.nereids.rules.RuleType; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteScan; - -import java.util.Optional; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnion; /** - * Implementation rule that convert logical Recursive Cte Scan to physical Recursive Cte Scan. + * Implementation rule that convert LogicalRecursiveUnion to PhysicalRecursiveUnion. */ -public class LogicalRecursiveCteScanToPhysicalRecursiveCteScan extends OneImplementationRuleFactory { +public class LogicalRecursiveUnionToPhysicalRecursiveUnion extends OneImplementationRuleFactory { @Override public Rule build() { - return logicalRecursiveCteScan().then(recursiveCteScan -> - new PhysicalRecursiveCteScan( - recursiveCteScan.getRelationId(), - recursiveCteScan.getTable(), - recursiveCteScan.getQualifier(), - Optional.empty(), - recursiveCteScan.getLogicalProperties(), - recursiveCteScan.getOperativeSlots()) - ).toRule(RuleType.LOGICAL_RECURSIVE_CTE_SCAN_TO_PHYSICAL_RECUSIVE_CTE_SCAN_RULE); + return logicalRecursiveUnion().then(recursiveCte -> new PhysicalRecursiveUnion(recursiveCte.getCteName(), + recursiveCte.getQualifier(), + recursiveCte.getOutputs(), + recursiveCte.getRegularChildrenOutputs(), + recursiveCte.getLogicalProperties(), + recursiveCte.left(), + recursiveCte.right())).toRule(RuleType.LOGICAL_RECURSIVE_UNION_TO_PHYSICAL_RECURSIVE_UNION); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWorkTableReferenceToPhysicalWorkTableReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWorkTableReferenceToPhysicalWorkTableReference.java new file mode 100644 index 00000000000000..430b825365371a --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/implementation/LogicalWorkTableReferenceToPhysicalWorkTableReference.java @@ -0,0 +1,41 @@ +// 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.physical.PhysicalWorkTableReference; + +import java.util.Optional; + +/** + * Implementation rule that convert LogicalWorkTableReference to PhysicalWorkTableReference. + */ +public class LogicalWorkTableReferenceToPhysicalWorkTableReference extends OneImplementationRuleFactory { + @Override + public Rule build() { + return logicalWorkTableReference().then(workTableReference -> new PhysicalWorkTableReference( + workTableReference.getRelationId(), + workTableReference.getCteId(), + workTableReference.getOutput(), + workTableReference.getNameParts(), + Optional.empty(), + workTableReference.getLogicalProperties())) + .toRule(RuleType.LOGICAL_WORK_TABLE_REFERENCE_TO_PHYSICAL_WORK_TABLE_REFERENCE); + } +} 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 204a018fbc7cb5..8592d3f22a0294 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 @@ -41,7 +41,6 @@ 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.LogicalRecursiveCte; 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.LogicalSink; @@ -290,53 +289,6 @@ public Plan visitLogicalRepeat(LogicalRepeat repeat, Map replaceMap) { - recursiveCte = (LogicalRecursiveCte) super.visit(recursiveCte, replaceMap); - ImmutableList.Builder> newChildrenOutputs = ImmutableList.builder(); - List inputNullable = null; - if (!recursiveCte.children().isEmpty()) { - inputNullable = Lists.newArrayListWithCapacity(recursiveCte.getOutputs().size()); - for (int i = 0; i < recursiveCte.getOutputs().size(); i++) { - inputNullable.add(false); - } - for (int i = 0; i < recursiveCte.arity(); i++) { - List childOutput = recursiveCte.child(i).getOutput(); - List setChildOutput = recursiveCte.getRegularChildOutput(i); - ImmutableList.Builder newChildOutputs = ImmutableList.builder(); - for (int j = 0; j < setChildOutput.size(); j++) { - for (Slot slot : childOutput) { - if (slot.getExprId().equals(setChildOutput.get(j).getExprId())) { - inputNullable.set(j, slot.nullable() || inputNullable.get(j)); - newChildOutputs.add((SlotReference) slot); - break; - } - } - } - newChildrenOutputs.add(newChildOutputs.build()); - } - } - if (inputNullable == null) { - // this is a fail-safe - // means there is no children and having no getConstantExprsList - // no way to update the nullable flag, so just do nothing - return recursiveCte; - } - List outputs = recursiveCte.getOutputs(); - List newOutputs = Lists.newArrayListWithCapacity(outputs.size()); - for (int i = 0; i < inputNullable.size(); i++) { - NamedExpression ne = outputs.get(i); - Slot slot = ne instanceof Alias ? (Slot) ((Alias) ne).child() : (Slot) ne; - slot = slot.withNullable(inputNullable.get(i)); - NamedExpression newOutput = ne instanceof Alias ? (NamedExpression) ne.withChildren(slot) : slot; - newOutputs.add(newOutput); - replaceMap.put(newOutput.getExprId(), newOutput.toSlot()); - } - return recursiveCte.withNewOutputs(newOutputs) - .withChildrenAndTheirOutputs(recursiveCte.children(), newChildrenOutputs.build()) - .recomputeLogicalProperties(); - } - @Override public Plan visitLogicalSetOperation(LogicalSetOperation setOperation, Map replaceMap) { setOperation = (LogicalSetOperation) super.visit(setOperation, replaceMap); 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 index 9bbcb2e1e8d131..16c5d3d2c9d86f 100644 --- 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 @@ -21,6 +21,7 @@ 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.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; @@ -31,17 +32,14 @@ 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.logical.LogicalRecursiveCteRecursiveChild; 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 com.google.common.collect.Sets; import java.util.ArrayList; -import java.util.HashSet; import java.util.List; import java.util.Set; @@ -53,11 +51,11 @@ */ public class CTEInline extends DefaultPlanRewriter> implements CustomRewriter { // all cte used by recursive cte's recursive child should be inline - private Set mustInlineCteConsumers = new HashSet<>(); + private Set mustInlineCTEs; @Override public Plan rewriteRoot(Plan plan, JobContext jobContext) { - collectMustInlineCteConsumers(plan, false, mustInlineCteConsumers); + mustInlineCTEs = jobContext.getCascadesContext().getStatementContext().getMustInlineCTEs(); Plan root = plan.accept(this, null); // collect cte id to consumer @@ -86,7 +84,7 @@ public Plan visitLogicalCTEAnchor(LogicalCTEAnchor) cteAnchor.left()); // process child @@ -128,19 +126,4 @@ public Plan visitLogicalCTEConsumer(LogicalCTEConsumer cteConsumer, LogicalCTEPr } return cteConsumer; } - - private void collectMustInlineCteConsumers(Plan planNode, boolean needCollect, - Set cteConsumers) { - if (planNode instanceof LogicalCTEConsumer) { - if (needCollect) { - cteConsumers.add((LogicalCTEConsumer) planNode); - } - } else if (planNode instanceof LogicalRecursiveCteRecursiveChild) { - collectMustInlineCteConsumers(planNode.child(0), true, cteConsumers); - } else { - for (Plan child : planNode.children()) { - collectMustInlineCteConsumers(child, needCollect, cteConsumers); - } - } - } } 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 d92dea439dd4c3..cc2eb7d2057dce 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 @@ -44,7 +44,6 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalIntersect; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCte; 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; @@ -214,12 +213,6 @@ public Plan visitLogicalProject(LogicalProject project, PruneCon return pruneChildren(plan, new RoaringBitmap()); } - @Override - public Plan visitLogicalRecursiveCte(LogicalRecursiveCte recursiveCte, PruneContext context) { - // keep LogicalRecursiveCte's output unchanged - return skipPruneThis(recursiveCte); - } - // union can not prune children by the common logic, we must override visit method to write special code. @Override public Plan visitLogicalUnion(LogicalUnion union, PruneContext context) { @@ -410,54 +403,6 @@ public

P pruneOutput(P plan, List originOutput } } - private LogicalRecursiveCte pruneRecursiveCteOutput(LogicalRecursiveCte recursiveCte, PruneContext context) { - List originOutput = recursiveCte.getOutputs(); - if (originOutput.isEmpty()) { - return recursiveCte; - } - List prunedOutputs = Lists.newArrayList(); - List> regularChildrenOutputs = recursiveCte.getRegularChildrenOutputs(); - List children = recursiveCte.children(); - List extractColumnIndex = Lists.newArrayList(); - for (int i = 0; i < originOutput.size(); i++) { - NamedExpression output = originOutput.get(i); - if (context.requiredSlotsIds.contains(output.getExprId().asInt())) { - prunedOutputs.add(output); - extractColumnIndex.add(i); - } - } - - if (prunedOutputs.isEmpty()) { - // process prune all columns - NamedExpression originSlot = originOutput.get(0); - prunedOutputs = ImmutableList.of(new SlotReference(originSlot.getExprId(), originSlot.getName(), - TinyIntType.INSTANCE, false, originSlot.getQualifier())); - regularChildrenOutputs = Lists.newArrayListWithCapacity(regularChildrenOutputs.size()); - children = Lists.newArrayListWithCapacity(children.size()); - for (int i = 0; i < recursiveCte.children().size(); i++) { - Plan child = recursiveCte.child(i); - List newProjectOutput = ImmutableList.of(new Alias(new TinyIntLiteral((byte) 1))); - LogicalProject project; - if (child instanceof LogicalProject) { - LogicalProject childProject = (LogicalProject) child; - List mergeProjections = PlanUtils.mergeProjections( - childProject.getProjects(), newProjectOutput); - project = new LogicalProject<>(mergeProjections, childProject.child()); - } else { - project = new LogicalProject<>(newProjectOutput, child); - } - regularChildrenOutputs.add((List) project.getOutput()); - children.add(project); - } - } - - if (prunedOutputs.equals(originOutput) && !context.requiredSlotsIds.isEmpty()) { - return recursiveCte; - } else { - return recursiveCte.withNewOutputsAndChildren(prunedOutputs, children, regularChildrenOutputs); - } - } - private LogicalUnion pruneUnionOutput(LogicalUnion union, PruneContext context) { List originOutput = union.getOutputs(); if (originOutput.isEmpty()) { 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 7f112c4b4f8d0e..353cd4922120c0 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 @@ -91,9 +91,9 @@ 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.LogicalRecursiveCte; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCteRecursiveChild; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCteScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnion; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnionAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnionProducer; 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; @@ -102,6 +102,7 @@ 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.logical.LogicalWorkTableReference; import org.apache.doris.nereids.trees.plans.physical.PhysicalAssertNumRows; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEAnchor; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEConsumer; @@ -127,9 +128,9 @@ 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.PhysicalRecursiveCte; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteRecursiveChild; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteScan; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnion; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionAnchor; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionProducer; 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; @@ -139,6 +140,7 @@ 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; +import org.apache.doris.nereids.trees.plans.physical.PhysicalWorkTableReference; import org.apache.doris.nereids.trees.plans.visitor.DefaultPlanVisitor; import org.apache.doris.nereids.types.DataType; import org.apache.doris.qe.ConnectContext; @@ -183,6 +185,7 @@ public class StatsCalculator extends DefaultPlanVisitor { public static double DEFAULT_AGGREGATE_RATIO = 1 / 3.0; public static double AGGREGATE_COLUMN_CORRELATION_COEFFICIENT = 0.75; public static double DEFAULT_COLUMN_NDV_RATIO = 0.5; + public static double RECURSIVE_CTE_EXPAND_RATIO = 5.0; protected static final Logger LOG = LogManager.getLogger(StatsCalculator.class); protected final GroupExpression groupExpression; @@ -884,9 +887,27 @@ public Statistics visitLogicalEsScan(LogicalEsScan esScan, Void context) { } @Override - public Statistics visitLogicalRecursiveCteScan(LogicalRecursiveCteScan recursiveCteScan, Void context) { - recursiveCteScan.getExpressions(); - return computeCatalogRelation(recursiveCteScan); + public Statistics visitLogicalWorkTableReference(LogicalWorkTableReference workTableReference, Void context) { + CTEId cteId = workTableReference.getCteId(); + Statistics prodStats = cteIdToStats.get(cteId); + Preconditions.checkArgument(prodStats != null, String.format("Stats for CTE: %s not found", cteId)); + Statistics consumerStats = new Statistics(prodStats.getRowCount(), 1, new HashMap<>()); + // because recursive cte's anchor outputs are same as worktable's output, we compare slot name for simplicity + Map columnStats = new HashMap<>(); + for (Map.Entry entry : prodStats.columnStatistics().entrySet()) { + columnStats.put(entry.getKey().getExpressionName(), entry.getValue()); + } + for (Slot slot : workTableReference.getOutput()) { + ColumnStatistic colStats = columnStats.get(slot.getName()); + if (colStats == null) { + continue; + } + consumerStats.addColumnStats(slot, colStats); + } + return new StatisticsBuilder() + .setRowCount(Math.max(1, consumerStats.getRowCount()) * RECURSIVE_CTE_EXPAND_RATIO) + .putColumnStatistics(consumerStats.columnStatistics()) + .build(); } @Override @@ -933,15 +954,24 @@ public Statistics visitLogicalAssertNumRows( } @Override - public Statistics visitLogicalRecursiveCte( - LogicalRecursiveCte recursiveCte, Void context) { + public Statistics visitLogicalRecursiveUnion( + LogicalRecursiveUnion recursiveCte, Void context) { return computeRecursiveCte(recursiveCte, groupExpression.children() .stream().map(Group::getStatistics).collect(Collectors.toList())); } @Override - public Statistics visitLogicalRecursiveCteRecursiveChild(LogicalRecursiveCteRecursiveChild recursiveChild, + public Statistics visitLogicalRecursiveUnionAnchor(LogicalRecursiveUnionAnchor recursiveAnchor, + Void context) { + StatisticsBuilder builder = new StatisticsBuilder(groupExpression.childStatistics(0)); + Statistics statistics = builder.setWidthInJoinCluster(1).build(); + cteIdToStats.put(recursiveAnchor.getCteId(), statistics); + return statistics; + } + + @Override + public Statistics visitLogicalRecursiveUnionProducer(LogicalRecursiveUnionProducer recursiveProducer, Void context) { return groupExpression.childStatistics(0); } @@ -1032,8 +1062,27 @@ public Statistics visitPhysicalSchemaScan(PhysicalSchemaScan schemaScan, Void co } @Override - public Statistics visitPhysicalRecursiveCteScan(PhysicalRecursiveCteScan recursiveCteScan, Void context) { - return computeCatalogRelation(recursiveCteScan); + public Statistics visitPhysicalWorkTableReference(PhysicalWorkTableReference workTableReference, Void context) { + CTEId cteId = workTableReference.getCteId(); + Statistics prodStats = cteIdToStats.get(cteId); + Preconditions.checkArgument(prodStats != null, String.format("Stats for CTE: %s not found", cteId)); + Statistics consumerStats = new Statistics(prodStats.getRowCount(), 1, new HashMap<>()); + // because recursive cte's anchor outputs are same as worktable's output, we compare slot name for simplicity + Map columnStats = new HashMap<>(); + for (Map.Entry entry : prodStats.columnStatistics().entrySet()) { + columnStats.put(entry.getKey().getExpressionName(), entry.getValue()); + } + for (Slot slot : workTableReference.getOutput()) { + ColumnStatistic colStats = columnStats.get(slot.getName()); + if (colStats == null) { + continue; + } + consumerStats.addColumnStats(slot, colStats); + } + return new StatisticsBuilder() + .setRowCount(Math.max(1, consumerStats.getRowCount()) * RECURSIVE_CTE_EXPAND_RATIO) + .putColumnStatistics(consumerStats.columnStatistics()) + .build(); } @Override @@ -1123,13 +1172,23 @@ public Statistics visitPhysicalAssertNumRows(PhysicalAssertNumRows recursiveUnion, + Void context) { + return computeRecursiveCte(recursiveUnion, groupExpression.children() .stream().map(Group::getStatistics).collect(Collectors.toList())); } @Override - public Statistics visitPhysicalRecursiveCteRecursiveChild(PhysicalRecursiveCteRecursiveChild recursiveChild, + public Statistics visitPhysicalRecursiveUnionAnchor(PhysicalRecursiveUnionAnchor recursiveUnionAnchor, + Void context) { + StatisticsBuilder builder = new StatisticsBuilder(groupExpression.childStatistics(0)); + Statistics statistics = builder.setWidthInJoinCluster(1).build(); + cteIdToStats.put(recursiveUnionAnchor.getCteId(), statistics); + return statistics; + } + + @Override + public Statistics visitPhysicalRecursiveUnionProducer(PhysicalRecursiveUnionProducer recursiveUnionProducer, Void context) { return groupExpression.childStatistics(0); } @@ -1519,7 +1578,7 @@ public Statistics computeEmptyRelation(EmptyRelation emptyRelation) { * computeRecursiveCte */ public Statistics computeRecursiveCte(RecursiveCte recursiveCte, List childStats) { - // TODO: refactor this for one row relation + // similar as computeUnion List head; Statistics headStats; List> childOutputs = Lists.newArrayList(recursiveCte.getRegularChildrenOutputs()); 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 237ec5c69300ea..05fb4bccc67119 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 @@ -53,8 +53,9 @@ 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.LogicalRecursiveCte; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCteRecursiveChild; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnion; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnionAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnionProducer; 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.LogicalSink; @@ -365,7 +366,8 @@ public Plan visitLogicalUnion(LogicalUnion union, DeepCopierContext context) { } @Override - public Plan visitLogicalRecursiveCte(LogicalRecursiveCte recursiveCte, DeepCopierContext context) { + public Plan visitLogicalRecursiveUnion(LogicalRecursiveUnion recursiveCte, + DeepCopierContext context) { List children = recursiveCte.children().stream() .map(c -> c.accept(this, context)) .collect(ImmutableList.toImmutableList()); @@ -377,15 +379,22 @@ public Plan visitLogicalRecursiveCte(LogicalRecursiveCte recursiveCte, DeepCopie .map(o -> (SlotReference) ExpressionDeepCopier.INSTANCE.deepCopy(o, context)) .collect(ImmutableList.toImmutableList())) .collect(ImmutableList.toImmutableList()); - return new LogicalRecursiveCte(recursiveCte.getCteName(), recursiveCte.isUnionAll(), outputs, + return new LogicalRecursiveUnion(recursiveCte.getCteName(), recursiveCte.getQualifier(), outputs, childrenOutputs, children); } @Override - public Plan visitLogicalRecursiveCteRecursiveChild(LogicalRecursiveCteRecursiveChild recursiveChild, + public Plan visitLogicalRecursiveUnionAnchor(LogicalRecursiveUnionAnchor recursiveAnchor, + DeepCopierContext context) { + Plan child = recursiveAnchor.child().accept(this, context); + return new LogicalRecursiveUnionAnchor<>(recursiveAnchor.getCteId(), child); + } + + @Override + public Plan visitLogicalRecursiveUnionProducer(LogicalRecursiveUnionProducer recursiveProducer, DeepCopierContext context) { - Plan child = recursiveChild.child().accept(this, context); - return new LogicalRecursiveCteRecursiveChild<>(recursiveChild.getCteName(), child); + Plan child = recursiveProducer.child().accept(this, context); + return new LogicalRecursiveUnionProducer<>(recursiveProducer.getCteName(), child); } @Override 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 08d7c3bb7865de..467c70a1502022 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 @@ -85,7 +85,7 @@ public enum PlanType { LOGICAL_QUALIFY, LOGICAL_RECURSIVE_CTE, LOGICAL_RECURSIVE_CTE_RECURSIVE_CHILD, - LOGICAL_RECURSIVE_CTE_SCAN, + LOGICAL_WORK_TABLE_REFERENCE, LOGICAL_REPEAT, LOGICAL_SELECT_HINT, LOGICAL_SUBQUERY_ALIAS, @@ -110,7 +110,7 @@ public enum PlanType { PHYSICAL_OLAP_SCAN, PHYSICAL_SCHEMA_SCAN, PHYSICAL_TVF_RELATION, - PHYSICAL_RECURSIVE_CTE_SCAN, + PHYSICAL_WORK_TABLE_REFERENCE, // physical sinks PHYSICAL_FILE_SINK, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java index 31f11a903f1b21..74f2bd1bfec007 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateMvByPartitionCommand.java @@ -269,7 +269,7 @@ public Plan visitLogicalCTE(LogicalCTE cte, PredicateAddContext ); rewrittenSubQueryAlias.add(subQueryAlias.withChildren(subQueryAliasChildren)); } - return super.visitLogicalCTE(new LogicalCTE<>(cte.isRecursiveCte(), + return super.visitLogicalCTE(new LogicalCTE<>(cte.isRecursive(), rewrittenSubQueryAlias, cte.child()), predicates); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java index 0cf40d31cbb184..d795002215182a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedJobBuilder.java @@ -29,7 +29,6 @@ import org.apache.doris.planner.PlanFragment; import org.apache.doris.planner.PlanFragmentId; import org.apache.doris.planner.PlanNodeId; -import org.apache.doris.planner.RecursiveCteScanNode; import org.apache.doris.planner.ScanNode; import org.apache.doris.planner.SchemaScanNode; import org.apache.doris.thrift.TExplainLevel; @@ -127,10 +126,6 @@ private UnassignedJob buildLeafOrScanJob( unassignedJob = buildScanMetadataJob( statementContext, planFragment, (SchemaScanNode) scanNode, scanWorkerSelector ); - } else if (scanNode instanceof RecursiveCteScanNode) { - unassignedJob = buildScanRecursiveCteJob( - statementContext, planFragment, (RecursiveCteScanNode) scanNode, inputJobs, scanWorkerSelector - ); } else { // only scan external tables or cloud tables or table valued functions // e,g. select * from numbers('number'='100') @@ -201,14 +196,6 @@ private UnassignedJob buildScanMetadataJob( return new UnassignedScanMetadataJob(statementContext, fragment, schemaScanNode, scanWorkerSelector); } - private UnassignedJob buildScanRecursiveCteJob( - StatementContext statementContext, PlanFragment fragment, - RecursiveCteScanNode recursiveCteScanNode, - ListMultimap inputJobs, ScanWorkerSelector scanWorkerSelector) { - return new UnassignedRecursiveCteScanJob(statementContext, fragment, recursiveCteScanNode, - inputJobs, scanWorkerSelector); - } - private UnassignedJob buildScanRemoteTableJob( StatementContext statementContext, PlanFragment planFragment, List scanNodes, ListMultimap inputJobs, diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedRecursiveCteScanJob.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedRecursiveCteScanJob.java deleted file mode 100644 index cfd3ebe7b2b336..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/distribute/worker/job/UnassignedRecursiveCteScanJob.java +++ /dev/null @@ -1,65 +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.distribute.worker.job; - -import org.apache.doris.nereids.StatementContext; -import org.apache.doris.nereids.trees.plans.distribute.DistributeContext; -import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorker; -import org.apache.doris.nereids.trees.plans.distribute.worker.DistributedPlanWorkerManager; -import org.apache.doris.nereids.trees.plans.distribute.worker.ScanWorkerSelector; -import org.apache.doris.planner.ExchangeNode; -import org.apache.doris.planner.PlanFragment; -import org.apache.doris.planner.ScanNode; - -import com.google.common.base.Preconditions; -import com.google.common.collect.ImmutableList; -import com.google.common.collect.ListMultimap; - -import java.util.List; -import java.util.Map; -import java.util.Objects; - -/** - * UnassignedRecursiveCteScanJob - */ -public class UnassignedRecursiveCteScanJob extends AbstractUnassignedScanJob { - private final ScanWorkerSelector scanWorkerSelector; - - public UnassignedRecursiveCteScanJob( - StatementContext statementContext, PlanFragment fragment, ScanNode scanNode, - ListMultimap exchangeToChildJob, ScanWorkerSelector scanWorkerSelector) { - super(statementContext, fragment, ImmutableList.of(scanNode), exchangeToChildJob); - this.scanWorkerSelector = Objects.requireNonNull(scanWorkerSelector, "scanWorkerSelector is not null"); - } - - @Override - protected Map multipleMachinesParallelization( - DistributeContext distributeContext, ListMultimap inputJobs) { - return scanWorkerSelector.selectReplicaAndWorkerWithoutBucket( - scanNodes.get(0), statementContext.getConnectContext() - ); - } - - @Override - protected List fillUpAssignedJobs(List assignedJobs, - DistributedPlanWorkerManager workerManager, ListMultimap inputJobs) { - Preconditions.checkArgument(!assignedJobs.isEmpty(), - "assignedJobs is empty for UnassignedRecursiveCteScanJob"); - return assignedJobs; - } -} 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 5ceb0e55bca5e2..00229afa10d03f 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 @@ -41,26 +41,26 @@ public class LogicalCTE extends LogicalUnary implements PropagateFuncDeps { private final List> aliasQueries; - private final boolean isRecursiveCte; + private final boolean isRecursive; - public LogicalCTE(boolean isRecursiveCte, List> aliasQueries, CHILD_TYPE child) { - this(isRecursiveCte, aliasQueries, Optional.empty(), Optional.empty(), child); + public LogicalCTE(boolean isRecursive, List> aliasQueries, CHILD_TYPE child) { + this(isRecursive, aliasQueries, Optional.empty(), Optional.empty(), child); } - public LogicalCTE(boolean isRecursiveCte, List> aliasQueries, + public LogicalCTE(boolean isRecursive, List> aliasQueries, Optional groupExpression, 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.isRecursiveCte = isRecursiveCte; + this.isRecursive = isRecursive; } public List> getAliasQueries() { return aliasQueries; } - public boolean isRecursiveCte() { - return isRecursiveCte; + public boolean isRecursive() { + return isRecursive; } @Override @@ -81,7 +81,7 @@ public List computeOutput() { @Override public String toString() { return Utils.toSqlString("LogicalCTE", - "isRecursiveCte", isRecursiveCte, + "isRecursive", isRecursive, "aliasQueries", aliasQueries ); } @@ -113,18 +113,18 @@ public boolean equals(Object o) { return false; } LogicalCTE that = (LogicalCTE) o; - return aliasQueries.equals(that.aliasQueries) && isRecursiveCte == that.isRecursiveCte; + return aliasQueries.equals(that.aliasQueries) && isRecursive == that.isRecursive; } @Override public int hashCode() { - return Objects.hash(aliasQueries, isRecursiveCte); + return Objects.hash(aliasQueries, isRecursive); } @Override public Plan withChildren(List children) { Preconditions.checkArgument(aliasQueries.size() > 0); - return new LogicalCTE<>(isRecursiveCte, aliasQueries, children.get(0)); + return new LogicalCTE<>(isRecursive, aliasQueries, children.get(0)); } @Override @@ -139,7 +139,7 @@ public List getExpressions() { @Override public LogicalCTE withGroupExpression(Optional groupExpression) { - return new LogicalCTE<>(isRecursiveCte, aliasQueries, groupExpression, + return new LogicalCTE<>(isRecursive, aliasQueries, groupExpression, Optional.of(getLogicalProperties()), child()); } @@ -147,6 +147,6 @@ public LogicalCTE withGroupExpression(Optional grou public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { Preconditions.checkArgument(aliasQueries.size() > 0); - return new LogicalCTE<>(isRecursiveCte, aliasQueries, groupExpression, logicalProperties, children.get(0)); + return new LogicalCTE<>(isRecursive, aliasQueries, groupExpression, logicalProperties, children.get(0)); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCteScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCteScan.java deleted file mode 100644 index 31b335260b9a29..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCteScan.java +++ /dev/null @@ -1,87 +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.logical; - -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.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 java.util.List; -import java.util.Optional; - -/** - * LogicalRecursiveCteScan. - */ -public class LogicalRecursiveCteScan extends LogicalCatalogRelation { - public LogicalRecursiveCteScan(RelationId relationId, TableIf table, List qualifier) { - this(relationId, table, qualifier, Optional.empty(), Optional.empty()); - } - - private LogicalRecursiveCteScan(RelationId relationId, TableIf table, List qualifier, - Optional groupExpression, Optional logicalProperties) { - super(relationId, PlanType.LOGICAL_RECURSIVE_CTE_SCAN, table, qualifier, groupExpression, logicalProperties); - } - - private LogicalRecursiveCteScan(RelationId relationId, TableIf table, List qualifier, - Optional groupExpression, Optional logicalProperties, - String tableAlias) { - super(relationId, PlanType.LOGICAL_RECURSIVE_CTE_SCAN, table, qualifier, groupExpression, - logicalProperties, tableAlias); - } - - @Override - public String toString() { - return Utils.toSqlString("LogicalRecursiveCteScan", - "cteName", table.getName()); - } - - @Override - public Plan withGroupExpression(Optional groupExpression) { - return new LogicalRecursiveCteScan(relationId, table, qualifier, - groupExpression, Optional.ofNullable(getLogicalProperties()), tableAlias); - } - - @Override - public Plan withGroupExprLogicalPropChildren(Optional groupExpression, - Optional logicalProperties, List children) { - return new LogicalRecursiveCteScan(relationId, table, qualifier, groupExpression, logicalProperties, - tableAlias); - } - - @Override - public LogicalCatalogRelation withRelationId(RelationId relationId) { - return new LogicalRecursiveCteScan(relationId, table, qualifier, - groupExpression, Optional.ofNullable(getLogicalProperties()), tableAlias); - } - - @Override - public LogicalCatalogRelation withTableAlias(String tableAlias) { - return new LogicalRecursiveCteScan(relationId, table, qualifier, - groupExpression, Optional.ofNullable(getLogicalProperties()), tableAlias); - } - - @Override - public R accept(PlanVisitor visitor, C context) { - return visitor.visitLogicalRecursiveCteScan(this, context); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCte.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnion.java similarity index 50% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCte.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnion.java index bef00db5f72092..4c5ae7fd00cee5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCte.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnion.java @@ -17,10 +17,13 @@ package org.apache.doris.nereids.trees.plans.logical; +import org.apache.doris.nereids.CascadesContext; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.memo.GroupExpression; import org.apache.doris.nereids.properties.DataTrait; import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.properties.PhysicalProperties; +import org.apache.doris.nereids.rules.expression.ExpressionRewriteContext; import org.apache.doris.nereids.trees.expressions.Alias; import org.apache.doris.nereids.trees.expressions.Cast; import org.apache.doris.nereids.trees.expressions.ExprId; @@ -29,64 +32,90 @@ 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.Literal; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.algebra.RecursiveCte; +import org.apache.doris.nereids.trees.plans.algebra.SetOperation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.util.ExpressionUtils; import org.apache.doris.nereids.util.TypeCoercionUtils; import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.ConnectContext; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +import java.util.ArrayList; +import java.util.BitSet; +import java.util.HashMap; import java.util.List; +import java.util.Map; import java.util.Objects; import java.util.Optional; +import java.util.Set; /** - * LogicalRecursiveCte is basically like LogicalUnion + * LogicalRecursiveUnion is basically like LogicalUnion, so most methods are same or similar as LogicalUnion */ -public class LogicalRecursiveCte extends AbstractLogicalPlan implements RecursiveCte, OutputPrunable { +public class LogicalRecursiveUnion + extends LogicalBinary implements RecursiveCte { private final String cteName; private final List outputs; private final List> regularChildrenOutputs; - private final boolean isUnionAll; + private final SetOperation.Qualifier qualifier; - /** LogicalRecursiveCte */ - public LogicalRecursiveCte(String cteName, boolean isUnionAll, List children) { - this(cteName, isUnionAll, ImmutableList.of(), ImmutableList.of(), children); + /** LogicalRecursiveUnion */ + public LogicalRecursiveUnion(String cteName, SetOperation.Qualifier qualifier, List children) { + this(cteName, qualifier, ImmutableList.of(), ImmutableList.of(), children); } - /** LogicalRecursiveCte */ - public LogicalRecursiveCte(String cteName, boolean isUnionAll, List outputs, + /** LogicalRecursiveUnion */ + public LogicalRecursiveUnion(String cteName, SetOperation.Qualifier qualifier, List outputs, List> childrenOutputs, List children) { - this(cteName, isUnionAll, outputs, childrenOutputs, Optional.empty(), + this(cteName, qualifier, outputs, childrenOutputs, Optional.empty(), Optional.empty(), children); } - /** LogicalRecursiveCte */ - public LogicalRecursiveCte(String cteName, boolean isUnionAll, List outputs, + /** LogicalRecursiveUnion */ + public LogicalRecursiveUnion(String cteName, SetOperation.Qualifier qualifier, List outputs, List> childrenOutputs, - Optional groupExpression, Optional logicalProperties, + Optional groupExpression, + Optional logicalProperties, List children) { super(PlanType.LOGICAL_RECURSIVE_CTE, groupExpression, logicalProperties, children); this.cteName = cteName; - this.isUnionAll = isUnionAll; + this.qualifier = qualifier; this.outputs = ImmutableList.copyOf(outputs); this.regularChildrenOutputs = ImmutableList.copyOf(childrenOutputs); } + @Override + public LEFT_CHILD_TYPE left() { + return (LEFT_CHILD_TYPE) child(0); + } + + @Override + public RIGHT_CHILD_TYPE right() { + return (RIGHT_CHILD_TYPE) child(1); + } + @Override public boolean isUnionAll() { - return isUnionAll; + return qualifier == SetOperation.Qualifier.ALL; } public String getCteName() { return cteName; } + public SetOperation.Qualifier getQualifier() { + return qualifier; + } + @Override public List getRegularChildOutput(int i) { return regularChildrenOutputs.get(i); @@ -106,8 +135,7 @@ private List> castCommonDataTypeOutputs() { ImmutableList.Builder newLeftOutputs = ImmutableList.builderWithExpectedSize( childOutputSize); ImmutableList.Builder newRightOutputs = ImmutableList.builderWithExpectedSize( - childOutputSize - ); + childOutputSize); // Ensure that the output types of the left and right children are consistent and expand upward. for (int i = 0; i < childOutputSize; ++i) { Slot left = child(0).getOutput().get(i); @@ -146,8 +174,7 @@ public List buildNewOutputs() { Slot slot = slots.get(i); ExprId exprId = i < outputs.size() ? outputs.get(i).getExprId() : StatementScopeIdGenerator.newExprId(); newOutputs.add( - new SlotReference(exprId, slot.toSql(), slot.getDataType(), slot.nullable(), ImmutableList.of()) - ); + new SlotReference(exprId, slot.toSql(), slot.getDataType(), slot.nullable(), ImmutableList.of())); } return newOutputs.build(); } @@ -155,8 +182,8 @@ public List buildNewOutputs() { // If the right child is nullable, need to ensure that the left child is also nullable private List resetNullableForLeftOutputs() { int rightChildOutputSize = child(1).getOutput().size(); - ImmutableList.Builder resetNullableForLeftOutputs - = ImmutableList.builderWithExpectedSize(rightChildOutputSize); + ImmutableList.Builder resetNullableForLeftOutputs = ImmutableList + .builderWithExpectedSize(rightChildOutputSize); for (int i = 0; i < rightChildOutputSize; ++i) { if (child(1).getOutput().get(i).nullable() && !child(0).getOutput().get(i).nullable()) { resetNullableForLeftOutputs.add(child(0).getOutput().get(i).withNullable(true)); @@ -169,9 +196,9 @@ private List resetNullableForLeftOutputs() { @Override public String toString() { - return Utils.toSqlStringSkipNull("LogicalRecursiveCte", + return Utils.toSqlStringSkipNull("LogicalRecursiveUnion", "cteName", cteName, - "isUnionAll", isUnionAll, + "Qualifier", qualifier, "outputs", outputs, "regularChildrenOutputs", regularChildrenOutputs, "stats", statistics); @@ -185,19 +212,19 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - LogicalRecursiveCte that = (LogicalRecursiveCte) o; - return cteName.equals(that.cteName) && isUnionAll == that.isUnionAll && Objects.equals(outputs, that.outputs) + LogicalRecursiveUnion that = (LogicalRecursiveUnion) o; + return cteName.equals(that.cteName) && qualifier == that.qualifier && Objects.equals(outputs, that.outputs) && Objects.equals(regularChildrenOutputs, that.regularChildrenOutputs); } @Override public int hashCode() { - return Objects.hash(cteName, isUnionAll, outputs, regularChildrenOutputs); + return Objects.hash(cteName, qualifier, outputs, regularChildrenOutputs); } @Override public R accept(PlanVisitor visitor, C context) { - return visitor.visitLogicalRecursiveCte(this, context); + return visitor.visitLogicalRecursiveUnion(this, context); } @Override @@ -213,40 +240,40 @@ public List computeOutput() { } @Override - public LogicalRecursiveCte withChildren(List children) { - return new LogicalRecursiveCte(cteName, isUnionAll, outputs, regularChildrenOutputs, children); + public LogicalRecursiveUnion withChildren(List children) { + return new LogicalRecursiveUnion<>(cteName, qualifier, outputs, regularChildrenOutputs, children); } - public LogicalRecursiveCte withChildrenAndTheirOutputs(List children, + public LogicalRecursiveUnion withChildrenAndTheirOutputs(List children, List> childrenOutputs) { Preconditions.checkArgument(children.size() == childrenOutputs.size(), "children size %s is not equals with children outputs size %s", children.size(), childrenOutputs.size()); - return new LogicalRecursiveCte(cteName, isUnionAll, outputs, childrenOutputs, children); + return new LogicalRecursiveUnion<>(cteName, qualifier, outputs, childrenOutputs, children); } @Override - public LogicalRecursiveCte withGroupExpression(Optional groupExpression) { - return new LogicalRecursiveCte(cteName, isUnionAll, outputs, regularChildrenOutputs, + public LogicalRecursiveUnion withGroupExpression(Optional groupExpression) { + return new LogicalRecursiveUnion<>(cteName, qualifier, outputs, regularChildrenOutputs, groupExpression, Optional.of(getLogicalProperties()), children); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalRecursiveCte(cteName, isUnionAll, outputs, regularChildrenOutputs, + return new LogicalRecursiveUnion<>(cteName, qualifier, outputs, regularChildrenOutputs, groupExpression, logicalProperties, children); } - public LogicalRecursiveCte withNewOutputs(List newOutputs) { - return new LogicalRecursiveCte(cteName, isUnionAll, newOutputs, regularChildrenOutputs, + public LogicalRecursiveUnion withNewOutputs(List newOutputs) { + return new LogicalRecursiveUnion<>(cteName, qualifier, newOutputs, regularChildrenOutputs, Optional.empty(), Optional.empty(), children); } - public LogicalRecursiveCte withNewOutputsAndChildren(List newOutputs, - List children, - List> childrenOutputs) { - return new LogicalRecursiveCte(cteName, isUnionAll, newOutputs, childrenOutputs, + public LogicalRecursiveUnion withNewOutputsAndChildren(List newOutputs, + List children, + List> childrenOutputs) { + return new LogicalRecursiveUnion<>(cteName, qualifier, newOutputs, childrenOutputs, Optional.empty(), Optional.empty(), children); } @@ -255,17 +282,45 @@ public List getOutputs() { return outputs; } - @Override - public LogicalRecursiveCte pruneOutputs(List prunedOutputs) { - return withNewOutputs(prunedOutputs); - } - @Override public void computeUnique(DataTrait.Builder builder) { + if (qualifier == SetOperation.Qualifier.DISTINCT) { + builder.addUniqueSlot(ImmutableSet.copyOf(getOutput())); + } } @Override public void computeUniform(DataTrait.Builder builder) { + final Optional context = ConnectContext.get() == null ? Optional.empty() + : Optional.of(new ExpressionRewriteContext(this, CascadesContext.initContext( + ConnectContext.get().getStatementContext(), this, PhysicalProperties.ANY))); + for (int i = 0; i < getOutputs().size(); i++) { + Optional value = Optional.empty(); + for (int childIdx = 0; childIdx < children.size(); childIdx++) { + List originOutputs = regularChildrenOutputs.get(childIdx); + Slot slot = originOutputs.get(i); + Optional childValue = child(childIdx).getLogicalProperties() + .getTrait().getUniformValue(slot); + if (childValue == null || !childValue.isPresent() || !childValue.get().isConstant()) { + value = Optional.empty(); + break; + } + Optional constExprOpt = ExpressionUtils.checkConstantExpr(childValue.get(), context); + if (!constExprOpt.isPresent()) { + value = Optional.empty(); + break; + } + if (!value.isPresent()) { + value = constExprOpt; + } else if (!value.equals(constExprOpt)) { + value = Optional.empty(); + break; + } + } + if (value.isPresent()) { + builder.addUniformSlotAndLiteral(getOutputs().get(i).toSlot(), value.get()); + } + } } @Override @@ -275,10 +330,77 @@ public boolean hasUnboundExpression() { @Override public void computeEqualSet(DataTrait.Builder builder) { + if (children.isEmpty()) { + return; + } + + // Get the list of equal slot sets and their corresponding index mappings for the first child + List> childEqualSlotsList = child(0).getLogicalProperties() + .getTrait().calAllEqualSet(); + List childEqualSlotsIndicesList = mapSlotToIndex(child(0), childEqualSlotsList); + List unionEqualSlotIndicesList = new ArrayList<>(childEqualSlotsIndicesList); + + // Traverse all children and find the equal sets that exist in all children + for (int i = 1; i < children.size(); i++) { + Plan child = children.get(i); + + // Get the equal slot sets for the current child + childEqualSlotsList = child.getLogicalProperties().getTrait().calAllEqualSet(); + + // Map slots to indices for the current child + childEqualSlotsIndicesList = mapSlotToIndex(child, childEqualSlotsList); + + // Only keep the equal pairs that exist in all children of the union + // This is done by calculating the intersection of all children's equal slot indices + for (BitSet unionEqualSlotIndices : unionEqualSlotIndicesList) { + BitSet intersect = new BitSet(); + for (BitSet childEqualSlotIndices : childEqualSlotsIndicesList) { + if (unionEqualSlotIndices.intersects(childEqualSlotIndices)) { + intersect = childEqualSlotIndices; + break; + } + } + unionEqualSlotIndices.and(intersect); + } + } + + // Build the functional dependencies for the output slots + List outputList = getOutput(); + for (BitSet equalSlotIndices : unionEqualSlotIndicesList) { + if (equalSlotIndices.cardinality() <= 1) { + continue; + } + int first = equalSlotIndices.nextSetBit(0); + int next = equalSlotIndices.nextSetBit(first + 1); + while (next > 0) { + builder.addEqualPair(outputList.get(first), outputList.get(next)); + next = equalSlotIndices.nextSetBit(next + 1); + } + } } @Override public void computeFd(DataTrait.Builder builder) { // don't generate } + + private List mapSlotToIndex(Plan plan, List> equalSlotsList) { + Map slotToIndex = new HashMap<>(); + for (int i = 0; i < plan.getOutput().size(); i++) { + slotToIndex.put(plan.getOutput().get(i), i); + } + List equalSlotIndicesList = new ArrayList<>(); + for (Set equalSlots : equalSlotsList) { + BitSet equalSlotIndices = new BitSet(); + for (Slot slot : equalSlots) { + if (slotToIndex.containsKey(slot)) { + equalSlotIndices.set(slotToIndex.get(slot)); + } + } + if (equalSlotIndices.cardinality() > 1) { + equalSlotIndicesList.add(equalSlotIndices); + } + } + return equalSlotIndicesList; + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnionAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnionAnchor.java new file mode 100644 index 00000000000000..96c3f812432c6c --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnionAnchor.java @@ -0,0 +1,136 @@ +// 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.DataTrait; +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; +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.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * LogicalRecursiveUnionAnchor is sentinel plan for anchor side in stats calculator + */ +public class LogicalRecursiveUnionAnchor extends LogicalUnary { + private final CTEId cteId; + + public LogicalRecursiveUnionAnchor(CTEId cteId, CHILD_TYPE child) { + this(cteId, Optional.empty(), Optional.empty(), child); + } + + public LogicalRecursiveUnionAnchor(CTEId cteId, Optional groupExpression, + Optional logicalProperties, CHILD_TYPE child) { + this(cteId, groupExpression, logicalProperties, ImmutableList.of(child)); + } + + public LogicalRecursiveUnionAnchor(CTEId cteId, Optional groupExpression, + Optional logicalProperties, List child) { + super(PlanType.LOGICAL_RECURSIVE_CTE_RECURSIVE_CHILD, groupExpression, logicalProperties, child); + this.cteId = cteId; + } + + public CTEId getCteId() { + return cteId; + } + + @Override + public Plan withChildren(List children) { + return new LogicalRecursiveUnionAnchor<>(cteId, Optional.empty(), Optional.empty(), children); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalRecursiveUnionAnchor(this, context); + } + + @Override + public List getExpressions() { + return ImmutableList.of(); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new LogicalRecursiveUnionAnchor<>(cteId, groupExpression, + Optional.of(getLogicalProperties()), children); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + return new LogicalRecursiveUnionAnchor<>(cteId, groupExpression, logicalProperties, children); + } + + @Override + public String toString() { + return Utils.toSqlStringSkipNull("LogicalRecursiveUnionAnchor", + "CTEId", cteId); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogicalRecursiveUnionAnchor that = (LogicalRecursiveUnionAnchor) o; + return cteId.equals(that.cteId); + } + + @Override + public int hashCode() { + return Objects.hash(cteId); + } + + @Override + public void computeUnique(DataTrait.Builder builder) { + + } + + @Override + public void computeUniform(DataTrait.Builder builder) { + + } + + @Override + public void computeEqualSet(DataTrait.Builder builder) { + + } + + @Override + public void computeFd(DataTrait.Builder builder) { + + } + + @Override + public List computeOutput() { + return child().getOutput(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCteRecursiveChild.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnionProducer.java similarity index 70% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCteRecursiveChild.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnionProducer.java index d2766a86f95acf..68bc22e2ce7c8f 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveCteRecursiveChild.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalRecursiveUnionProducer.java @@ -30,24 +30,25 @@ import com.google.common.collect.ImmutableList; import java.util.List; +import java.util.Objects; import java.util.Optional; /** - * LogicalRecursiveCteRecursiveChild is sentinel plan for must_shuffle + * LogicalRecursiveUnionProducer is sentinel plan for must_shuffle */ -public class LogicalRecursiveCteRecursiveChild extends LogicalUnary { +public class LogicalRecursiveUnionProducer extends LogicalUnary { private final String cteName; - public LogicalRecursiveCteRecursiveChild(String cteName, CHILD_TYPE child) { + public LogicalRecursiveUnionProducer(String cteName, CHILD_TYPE child) { this(cteName, Optional.empty(), Optional.empty(), child); } - public LogicalRecursiveCteRecursiveChild(String cteName, Optional groupExpression, + public LogicalRecursiveUnionProducer(String cteName, Optional groupExpression, Optional logicalProperties, CHILD_TYPE child) { this(cteName, groupExpression, logicalProperties, ImmutableList.of(child)); } - public LogicalRecursiveCteRecursiveChild(String cteName, Optional groupExpression, + public LogicalRecursiveUnionProducer(String cteName, Optional groupExpression, Optional logicalProperties, List child) { super(PlanType.LOGICAL_RECURSIVE_CTE_RECURSIVE_CHILD, groupExpression, logicalProperties, child); this.cteName = cteName; @@ -59,12 +60,12 @@ public String getCteName() { @Override public Plan withChildren(List children) { - return new LogicalRecursiveCteRecursiveChild<>(cteName, Optional.empty(), Optional.empty(), children); + return new LogicalRecursiveUnionProducer<>(cteName, Optional.empty(), Optional.empty(), children); } @Override public R accept(PlanVisitor visitor, C context) { - return visitor.visitLogicalRecursiveCteRecursiveChild(this, context); + return visitor.visitLogicalRecursiveUnionProducer(this, context); } @Override @@ -74,22 +75,39 @@ public List getExpressions() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new LogicalRecursiveCteRecursiveChild<>(cteName, groupExpression, + return new LogicalRecursiveUnionProducer<>(cteName, groupExpression, Optional.of(getLogicalProperties()), children); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { - return new LogicalRecursiveCteRecursiveChild<>(cteName, groupExpression, logicalProperties, children); + return new LogicalRecursiveUnionProducer<>(cteName, groupExpression, logicalProperties, children); } @Override public String toString() { - return Utils.toSqlStringSkipNull("LogicalRecursiveCteRecursiveChild", + return Utils.toSqlStringSkipNull("LogicalRecursiveUnionProducer", "cteName", cteName); } + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + LogicalRecursiveUnionProducer that = (LogicalRecursiveUnionProducer) o; + return cteName.equals(that.cteName); + } + + @Override + public int hashCode() { + return Objects.hash(cteName); + } + @Override public void computeUnique(DataTrait.Builder builder) { 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 8b18d3ce49d95b..4aff4dda9900f0 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 @@ -30,6 +30,7 @@ import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.LazyCompute; import org.apache.doris.nereids.util.Utils; +import org.apache.doris.qe.GlobalVariable; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -40,6 +41,7 @@ import java.util.ArrayList; import java.util.HashMap; import java.util.List; +import java.util.Locale; import java.util.Map; import java.util.Objects; import java.util.Optional; @@ -58,6 +60,7 @@ public class LogicalSubQueryAlias extends LogicalUnary< protected RelationId relationId; private final List qualifier; private final Optional> columnAliases; + // AnalyzeCTE will check this flag to deal with recursive and normal CTE respectively private final Supplier isRecursiveCte; public LogicalSubQueryAlias(String tableAlias, CHILD_TYPE child) { @@ -128,17 +131,40 @@ private List computeOutputInternal(boolean asteriskOutput) { private Supplier computeIsRecursiveCte() { return LazyCompute.of(() -> { - List relationList = collectToList(UnboundRelation.class::isInstance); + // we need check if any relation's name is same as alias query to know if it's recursive cte first + // so in later AnalyzeCTE, we could deal with recursive cte and normal cte separately + // it's a little ugly, maybe we can find a better way in future + List relationList = new ArrayList<>(8); + collectRelationsInCurrentCte(this, relationList); for (UnboundRelation relation : relationList) { List nameParts = relation.getNameParts(); - if (nameParts.size() == 1 && nameParts.get(0).equalsIgnoreCase(getAlias())) { - return true; + if (nameParts.size() == 1) { + String aliasName = getAlias(); + String tablename = nameParts.get(0); + if (GlobalVariable.lowerCaseTableNames != 0) { + aliasName = aliasName.toLowerCase(Locale.ROOT); + tablename = tablename.toLowerCase(Locale.ROOT); + } + if (aliasName.equals(tablename)) { + return true; + } } } return false; }); } + void collectRelationsInCurrentCte(Plan plan, List relationList) { + for (Plan child : plan.children()) { + if (child instanceof UnboundRelation) { + relationList.add((UnboundRelation) child); + } + if (!(child instanceof LogicalCTE)) { + collectRelationsInCurrentCte(child, relationList); + } + } + } + public boolean isRecursiveCte() { return isRecursiveCte.get(); } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWorkTableReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWorkTableReference.java new file mode 100644 index 00000000000000..f52bb20774d8a8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalWorkTableReference.java @@ -0,0 +1,149 @@ +// 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.DataTrait; +import org.apache.doris.nereids.properties.LogicalProperties; +import org.apache.doris.nereids.trees.expressions.CTEId; +import org.apache.doris.nereids.trees.expressions.Slot; +import org.apache.doris.nereids.trees.expressions.SlotReference; +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 com.google.common.collect.ImmutableList; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * LogicalWorkTableReference, recursive union's producer child will scan WorkTableReference in every iteration. + */ +public class LogicalWorkTableReference extends LogicalRelation { + private CTEId cteId; + private final List outputs; + private final List nameParts; + + public LogicalWorkTableReference(RelationId relationId, CTEId cteId, List outputs, List nameParts) { + this(relationId, cteId, outputs, nameParts, Optional.empty(), Optional.empty()); + } + + private LogicalWorkTableReference(RelationId relationId, CTEId cteId, List outputs, List nameParts, + Optional groupExpression, Optional logicalProperties) { + super(relationId, PlanType.LOGICAL_WORK_TABLE_REFERENCE, groupExpression, logicalProperties); + this.cteId = cteId; + this.outputs = Objects.requireNonNull(outputs); + this.nameParts = Objects.requireNonNull(nameParts); + } + + public List getNameParts() { + return nameParts; + } + + public String getTableName() { + return nameParts.stream().map(Utils::quoteIfNeeded) + .reduce((left, right) -> left + "." + right).orElse(""); + } + + public CTEId getCteId() { + return cteId; + } + + @Override + public String toString() { + return Utils.toSqlString("LogicalWorkTableReference", + "cteId", cteId, + "cteName", getTableName()); + } + + @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; + } + LogicalWorkTableReference that = (LogicalWorkTableReference) o; + return cteId.equals(that.cteId) && nameParts.equals(that.nameParts) && outputs.equals(that.outputs); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), cteId, nameParts, outputs); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new LogicalWorkTableReference(relationId, cteId, outputs, nameParts, + groupExpression, Optional.ofNullable(getLogicalProperties())); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + return new LogicalWorkTableReference(relationId, cteId, outputs, nameParts, groupExpression, logicalProperties); + } + + @Override + public void computeUnique(DataTrait.Builder builder) { + + } + + @Override + public void computeUniform(DataTrait.Builder builder) { + + } + + @Override + public void computeEqualSet(DataTrait.Builder builder) { + + } + + @Override + public void computeFd(DataTrait.Builder builder) { + + } + + @Override + public LogicalWorkTableReference withRelationId(RelationId relationId) { + return new LogicalWorkTableReference(relationId, cteId, outputs, nameParts, + groupExpression, Optional.ofNullable(getLogicalProperties())); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitLogicalWorkTableReference(this, context); + } + + @Override + public List computeOutput() { + ImmutableList.Builder slots = ImmutableList.builder(); + for (Slot slot : outputs) { + slots.add(new SlotReference(slot.getName(), slot.getDataType(), slot.nullable(), nameParts)); + } + return slots.build(); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCteScan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCteScan.java deleted file mode 100644 index 3450ae0de189b8..00000000000000 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCteScan.java +++ /dev/null @@ -1,85 +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.physical; - -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.properties.PhysicalProperties; -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.Utils; -import org.apache.doris.statistics.Statistics; - -import java.util.Collection; -import java.util.List; -import java.util.Optional; - -/** - * PhysicalRecursiveCteScan. - */ -public class PhysicalRecursiveCteScan extends PhysicalCatalogRelation { - public PhysicalRecursiveCteScan(RelationId relationId, TableIf table, List qualifier, - Optional groupExpression, LogicalProperties logicalProperties, - Collection operativeSlots) { - this(relationId, table, qualifier, groupExpression, logicalProperties, PhysicalProperties.ANY, null, - operativeSlots); - } - - public PhysicalRecursiveCteScan(RelationId relationId, TableIf table, List qualifier, - Optional groupExpression, LogicalProperties logicalProperties, - PhysicalProperties physicalProperties, Statistics statistics, Collection operativeSlots) { - super(relationId, PlanType.PHYSICAL_RECURSIVE_CTE_SCAN, table, qualifier, groupExpression, logicalProperties, - physicalProperties, statistics, operativeSlots); - } - - @Override - public R accept(PlanVisitor visitor, C context) { - return visitor.visitPhysicalRecursiveCteScan(this, context); - } - - @Override - public Plan withGroupExpression(Optional groupExpression) { - return new PhysicalRecursiveCteScan(relationId, table, qualifier, groupExpression, getLogicalProperties(), - physicalProperties, statistics, operativeSlots); - } - - @Override - public Plan withGroupExprLogicalPropChildren(Optional groupExpression, - Optional logicalProperties, List children) { - return new PhysicalRecursiveCteScan(relationId, table, qualifier, groupExpression, getLogicalProperties(), - physicalProperties, statistics, operativeSlots); - } - - @Override - public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalRecursiveCteScan(relationId, table, qualifier, groupExpression, getLogicalProperties(), - physicalProperties, statistics, operativeSlots); - } - - @Override - public String toString() { - return Utils.toSqlString("PhysicalRecursiveCteScan[" + table.getName() + "]" + getGroupIdWithPrefix(), - "stats", statistics, - "qualified", Utils.qualifiedName(qualifier, table.getName()), - "operativeCols", getOperativeSlots()); - } -} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCte.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnion.java similarity index 74% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCte.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnion.java index 44aab38fc304b1..ce5b074a60f04c 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCte.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnion.java @@ -28,6 +28,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.algebra.RecursiveCte; +import org.apache.doris.nereids.trees.plans.algebra.SetOperation; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.ConnectContext; @@ -46,50 +47,54 @@ import java.util.Set; /** - * PhysicalRecursiveCte is basically like PhysicalUnion + * PhysicalRecursiveUnion is basically like PhysicalUnion, so most methods are same or similar as PhysicalUnion */ -public class PhysicalRecursiveCte extends AbstractPhysicalPlan implements RecursiveCte { +public class PhysicalRecursiveUnion + extends PhysicalBinary implements RecursiveCte { private final String cteName; private final List outputs; private final List> regularChildrenOutputs; - private final boolean isUnionAll; + private final SetOperation.Qualifier qualifier; /** PhysicalRecursiveCte */ - public PhysicalRecursiveCte(String cteName, boolean isUnionAll, + public PhysicalRecursiveUnion(String cteName, SetOperation.Qualifier qualifier, List outputs, List> childrenOutputs, LogicalProperties logicalProperties, - List children) { - this(cteName, isUnionAll, outputs, childrenOutputs, Optional.empty(), logicalProperties, children); + LEFT_CHILD_TYPE leftChild, + RIGHT_CHILD_TYPE rightChild) { + this(cteName, qualifier, outputs, childrenOutputs, Optional.empty(), logicalProperties, leftChild, rightChild); } /** PhysicalRecursiveCte */ - public PhysicalRecursiveCte(String cteName, boolean isUnionAll, + public PhysicalRecursiveUnion(String cteName, SetOperation.Qualifier qualifier, List outputs, List> childrenOutputs, Optional groupExpression, LogicalProperties logicalProperties, - List children) { - this(cteName, isUnionAll, outputs, childrenOutputs, groupExpression, logicalProperties, - PhysicalProperties.ANY, null, children); + LEFT_CHILD_TYPE leftChild, + RIGHT_CHILD_TYPE rightChild) { + this(cteName, qualifier, outputs, childrenOutputs, groupExpression, logicalProperties, + PhysicalProperties.ANY, null, leftChild, rightChild); } /** PhysicalRecursiveCte */ - public PhysicalRecursiveCte(String cteName, boolean isUnionAll, List outputs, + public PhysicalRecursiveUnion(String cteName, SetOperation.Qualifier qualifier, List outputs, List> childrenOutputs, Optional groupExpression, LogicalProperties logicalProperties, - PhysicalProperties physicalProperties, Statistics statistics, List children) { + PhysicalProperties physicalProperties, Statistics statistics, LEFT_CHILD_TYPE leftChild, + RIGHT_CHILD_TYPE rightChild) { super(PlanType.PHYSICAL_RECURSIVE_CTE, groupExpression, logicalProperties, physicalProperties, - statistics, children.toArray(new Plan[0])); + statistics, leftChild, rightChild); this.cteName = cteName; - this.isUnionAll = isUnionAll; + this.qualifier = qualifier; this.outputs = ImmutableList.copyOf(outputs); this.regularChildrenOutputs = ImmutableList.copyOf(childrenOutputs); } @Override public boolean isUnionAll() { - return isUnionAll; + return qualifier == SetOperation.Qualifier.ALL; } public String getCteName() { @@ -126,19 +131,19 @@ public boolean equals(Object o) { if (o == null || getClass() != o.getClass()) { return false; } - PhysicalRecursiveCte that = (PhysicalRecursiveCte) o; - return cteName.equals(that.cteName) && isUnionAll == that.isUnionAll && Objects.equals(outputs, that.outputs) + PhysicalRecursiveUnion that = (PhysicalRecursiveUnion) o; + return cteName.equals(that.cteName) && qualifier == that.qualifier && Objects.equals(outputs, that.outputs) && Objects.equals(regularChildrenOutputs, that.regularChildrenOutputs); } @Override public int hashCode() { - return Objects.hash(cteName, isUnionAll, outputs, regularChildrenOutputs); + return Objects.hash(cteName, qualifier, outputs, regularChildrenOutputs); } @Override public R accept(PlanVisitor visitor, C context) { - return visitor.visitPhysicalRecursiveCte(this, context); + return visitor.visitPhysicalRecursiveUnion(this, context); } @Override @@ -151,7 +156,7 @@ public String toString() { return Utils.toSqlString("PhysicalRecursiveCte" + "[" + id.asInt() + "]" + getGroupIdWithPrefix(), "stats", statistics, "cteName", cteName, - "isUnionAll", isUnionAll, + "Qualifier", qualifier, "outputs", outputs, "regularChildrenOutputs", regularChildrenOutputs); } @@ -171,40 +176,41 @@ public String shapeInfo() { } @Override - public PhysicalRecursiveCte withChildren(List children) { - return new PhysicalRecursiveCte(cteName, isUnionAll, outputs, regularChildrenOutputs, groupExpression, - getLogicalProperties(), children); + public PhysicalRecursiveUnion withChildren(List children) { + return new PhysicalRecursiveUnion<>(cteName, qualifier, outputs, regularChildrenOutputs, groupExpression, + getLogicalProperties(), children.get(0), children.get(1)); } @Override - public PhysicalRecursiveCte withGroupExpression(Optional groupExpression) { - return new PhysicalRecursiveCte(cteName, isUnionAll, outputs, regularChildrenOutputs, - groupExpression, getLogicalProperties(), children); + public PhysicalRecursiveUnion withGroupExpression(Optional groupExpression) { + return new PhysicalRecursiveUnion<>(cteName, qualifier, outputs, regularChildrenOutputs, + groupExpression, getLogicalProperties(), left(), right()); } @Override - public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + public PhysicalRecursiveUnion withGroupExprLogicalPropChildren( + Optional groupExpression, Optional logicalProperties, List children) { - return new PhysicalRecursiveCte(cteName, isUnionAll, outputs, regularChildrenOutputs, - groupExpression, logicalProperties.get(), children); + return new PhysicalRecursiveUnion<>(cteName, qualifier, outputs, regularChildrenOutputs, + groupExpression, logicalProperties.get(), left(), right()); } @Override - public PhysicalRecursiveCte withPhysicalPropertiesAndStats( + public PhysicalRecursiveUnion withPhysicalPropertiesAndStats( PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalRecursiveCte(cteName, isUnionAll, outputs, regularChildrenOutputs, - groupExpression, getLogicalProperties(), physicalProperties, statistics, children); + return new PhysicalRecursiveUnion<>(cteName, qualifier, outputs, regularChildrenOutputs, + groupExpression, getLogicalProperties(), physicalProperties, statistics, left(), right()); } @Override - public PhysicalRecursiveCte resetLogicalProperties() { - return new PhysicalRecursiveCte(cteName, isUnionAll, outputs, regularChildrenOutputs, - Optional.empty(), null, physicalProperties, statistics, children); + public PhysicalRecursiveUnion resetLogicalProperties() { + return new PhysicalRecursiveUnion<>(cteName, qualifier, outputs, regularChildrenOutputs, + Optional.empty(), null, physicalProperties, statistics, left(), right()); } @Override public void computeUnique(DataTrait.Builder builder) { - if (!isUnionAll) { + if (qualifier == SetOperation.Qualifier.DISTINCT) { builder.addUniqueSlot(ImmutableSet.copyOf(getOutput())); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnionAnchor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnionAnchor.java new file mode 100644 index 00000000000000..1bef79ad50c8c6 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnionAnchor.java @@ -0,0 +1,144 @@ +// 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.DataTrait; +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.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 org.apache.doris.statistics.Statistics; + +import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; +import org.jetbrains.annotations.Nullable; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * PhysicalRecursiveUnionAnchor is sentinel plan for must_shuffle + */ +public class PhysicalRecursiveUnionAnchor extends PhysicalUnary { + private final CTEId cteId; + + public PhysicalRecursiveUnionAnchor(CTEId cteId, LogicalProperties logicalProperties, CHILD_TYPE child) { + this(cteId, Optional.empty(), logicalProperties, child); + } + + public PhysicalRecursiveUnionAnchor(CTEId cteId, Optional groupExpression, + LogicalProperties logicalProperties, CHILD_TYPE child) { + this(cteId, groupExpression, logicalProperties, PhysicalProperties.ANY, null, child); + } + + public PhysicalRecursiveUnionAnchor(CTEId cteId, Optional groupExpression, + LogicalProperties logicalProperties, @Nullable PhysicalProperties physicalProperties, Statistics statistics, + CHILD_TYPE child) { + super(PlanType.PHYSICAL_RECURSIVE_CTE_RECURSIVE_CHILD, groupExpression, logicalProperties, physicalProperties, + statistics, child); + this.cteId = cteId; + } + + public CTEId getCteId() { + return cteId; + } + + @Override + public String toString() { + return Utils.toSqlStringSkipNull("PhysicalRecursiveUnionAnchor", + "CTEId", cteId); + } + + @Override + public boolean equals(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + PhysicalRecursiveUnionAnchor that = (PhysicalRecursiveUnionAnchor) o; + return cteId.equals(that.cteId); + } + + @Override + public int hashCode() { + return Objects.hash(cteId); + } + + @Override + public Plan withChildren(List children) { + Preconditions.checkArgument(children.size() == 1); + return new PhysicalRecursiveUnionAnchor<>(cteId, groupExpression, getLogicalProperties(), + children.get(0)); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPhysicalRecursiveUnionAnchor(this, context); + } + + @Override + public List getExpressions() { + return ImmutableList.of(); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new PhysicalRecursiveUnionAnchor<>(cteId, groupExpression, getLogicalProperties(), child()); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + Preconditions.checkArgument(children.size() == 1); + return new PhysicalRecursiveUnionAnchor<>(cteId, groupExpression, logicalProperties.get(), child()); + } + + @Override + public void computeUnique(DataTrait.Builder builder) { + + } + + @Override + public void computeUniform(DataTrait.Builder builder) { + + } + + @Override + public void computeEqualSet(DataTrait.Builder builder) { + + } + + @Override + public void computeFd(DataTrait.Builder builder) { + + } + + @Override + public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { + return new PhysicalRecursiveUnionAnchor<>(cteId, groupExpression, getLogicalProperties(), + physicalProperties, statistics, child()); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCteRecursiveChild.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnionProducer.java similarity index 70% rename from fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCteRecursiveChild.java rename to fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnionProducer.java index 9aef71e7ee9404..fa7098a4dd692b 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveCteRecursiveChild.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalRecursiveUnionProducer.java @@ -33,24 +33,25 @@ import org.jetbrains.annotations.Nullable; import java.util.List; +import java.util.Objects; import java.util.Optional; /** - * PhysicalRecursiveCteRecursiveChild is sentinel plan for must_shuffle + * PhysicalRecursiveUnionProducer is sentinel plan for must_shuffle */ -public class PhysicalRecursiveCteRecursiveChild extends PhysicalUnary { +public class PhysicalRecursiveUnionProducer extends PhysicalUnary { private final String cteName; - public PhysicalRecursiveCteRecursiveChild(String cteName, LogicalProperties logicalProperties, CHILD_TYPE child) { + public PhysicalRecursiveUnionProducer(String cteName, LogicalProperties logicalProperties, CHILD_TYPE child) { this(cteName, Optional.empty(), logicalProperties, child); } - public PhysicalRecursiveCteRecursiveChild(String cteName, Optional groupExpression, + public PhysicalRecursiveUnionProducer(String cteName, Optional groupExpression, LogicalProperties logicalProperties, CHILD_TYPE child) { this(cteName, groupExpression, logicalProperties, PhysicalProperties.ANY, null, child); } - public PhysicalRecursiveCteRecursiveChild(String cteName, Optional groupExpression, + public PhysicalRecursiveUnionProducer(String cteName, Optional groupExpression, LogicalProperties logicalProperties, @Nullable PhysicalProperties physicalProperties, Statistics statistics, CHILD_TYPE child) { super(PlanType.PHYSICAL_RECURSIVE_CTE_RECURSIVE_CHILD, groupExpression, logicalProperties, physicalProperties, @@ -60,20 +61,37 @@ public PhysicalRecursiveCteRecursiveChild(String cteName, Optional children) { Preconditions.checkArgument(children.size() == 1); - return new PhysicalRecursiveCteRecursiveChild<>(cteName, groupExpression, getLogicalProperties(), + return new PhysicalRecursiveUnionProducer<>(cteName, groupExpression, getLogicalProperties(), children.get(0)); } @Override public R accept(PlanVisitor visitor, C context) { - return visitor.visitPhysicalRecursiveCteRecursiveChild(this, context); + return visitor.visitPhysicalRecursiveUnionProducer(this, context); } @Override @@ -83,14 +101,14 @@ public List getExpressions() { @Override public Plan withGroupExpression(Optional groupExpression) { - return new PhysicalRecursiveCteRecursiveChild<>(cteName, groupExpression, getLogicalProperties(), child()); + return new PhysicalRecursiveUnionProducer<>(cteName, groupExpression, getLogicalProperties(), child()); } @Override public Plan withGroupExprLogicalPropChildren(Optional groupExpression, Optional logicalProperties, List children) { Preconditions.checkArgument(children.size() == 1); - return new PhysicalRecursiveCteRecursiveChild<>(cteName, groupExpression, logicalProperties.get(), child()); + return new PhysicalRecursiveUnionProducer<>(cteName, groupExpression, logicalProperties.get(), child()); } @Override @@ -115,7 +133,7 @@ public void computeFd(DataTrait.Builder builder) { @Override public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { - return new PhysicalRecursiveCteRecursiveChild<>(cteName, groupExpression, getLogicalProperties(), + return new PhysicalRecursiveUnionProducer<>(cteName, groupExpression, getLogicalProperties(), physicalProperties, statistics, child()); } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWorkTableReference.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWorkTableReference.java new file mode 100644 index 00000000000000..83d31941c8c9c8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalWorkTableReference.java @@ -0,0 +1,152 @@ +// 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.DataTrait; +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.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.Utils; +import org.apache.doris.statistics.Statistics; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * PhysicalRecursiveCteScan. + */ +public class PhysicalWorkTableReference extends PhysicalRelation { + private CTEId cteId; + private final List outputs; + private final List nameParts; + + public PhysicalWorkTableReference(RelationId relationId, CTEId cteId, List outputs, List nameParts, + Optional groupExpression, LogicalProperties logicalProperties) { + this(relationId, cteId, outputs, nameParts, groupExpression, logicalProperties, PhysicalProperties.ANY, null); + } + + public PhysicalWorkTableReference(RelationId relationId, CTEId cteId, List outputs, List nameParts, + Optional groupExpression, LogicalProperties logicalProperties, + PhysicalProperties physicalProperties, Statistics statistics) { + super(relationId, PlanType.PHYSICAL_WORK_TABLE_REFERENCE, groupExpression, logicalProperties, + physicalProperties, statistics); + this.cteId = cteId; + this.outputs = Objects.requireNonNull(outputs); + this.nameParts = Objects.requireNonNull(nameParts); + } + + public CTEId getCteId() { + return cteId; + } + + public List getNameParts() { + return nameParts; + } + + public String getTableName() { + return nameParts.stream().map(Utils::quoteIfNeeded) + .reduce((left, right) -> left + "." + right).orElse(""); + } + + @Override + public R accept(PlanVisitor visitor, C context) { + return visitor.visitPhysicalWorkTableReference(this, context); + } + + @Override + public Plan withGroupExpression(Optional groupExpression) { + return new PhysicalWorkTableReference(relationId, cteId, outputs, nameParts, groupExpression, + getLogicalProperties(), + physicalProperties, statistics); + } + + @Override + public Plan withGroupExprLogicalPropChildren(Optional groupExpression, + Optional logicalProperties, List children) { + return new PhysicalWorkTableReference(relationId, cteId, outputs, nameParts, groupExpression, + getLogicalProperties(), + physicalProperties, statistics); + } + + @Override + public void computeUnique(DataTrait.Builder builder) { + + } + + @Override + public void computeUniform(DataTrait.Builder builder) { + + } + + @Override + public void computeEqualSet(DataTrait.Builder builder) { + + } + + @Override + public void computeFd(DataTrait.Builder builder) { + + } + + @Override + public PhysicalPlan withPhysicalPropertiesAndStats(PhysicalProperties physicalProperties, Statistics statistics) { + return new PhysicalWorkTableReference(relationId, cteId, outputs, nameParts, groupExpression, + getLogicalProperties(), + physicalProperties, statistics); + } + + @Override + public List getOutput() { + return outputs; + } + + @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; + } + PhysicalWorkTableReference that = (PhysicalWorkTableReference) o; + return cteId.equals(that.cteId) && nameParts.equals(that.nameParts) && outputs.equals(that.outputs); + } + + @Override + public int hashCode() { + return Objects.hash(super.hashCode(), cteId, nameParts, outputs); + } + + @Override + public String toString() { + return Utils.toSqlString("PhysicalWorkTableReference[" + getTableName() + "]" + getGroupIdWithPrefix(), + "stats", statistics, + "cteId", cteId, + "qualified", nameParts); + } +} 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 f2c9ec69bde22c..1aa50b53c05a36 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 @@ -45,8 +45,9 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalPreFilter; import org.apache.doris.nereids.trees.plans.logical.LogicalProject; import org.apache.doris.nereids.trees.plans.logical.LogicalQualify; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCte; -import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveCteRecursiveChild; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnion; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnionAnchor; +import org.apache.doris.nereids.trees.plans.logical.LogicalRecursiveUnionProducer; 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.LogicalSelectHint; @@ -81,8 +82,9 @@ 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.PhysicalRecursiveCte; -import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveCteRecursiveChild; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnion; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionAnchor; +import org.apache.doris.nereids.trees.plans.physical.PhysicalRecursiveUnionProducer; 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.PhysicalSetOperation; @@ -216,13 +218,19 @@ public R visitLogicalJoin(LogicalJoin join, C co return visit(join, context); } - public R visitLogicalRecursiveCte(LogicalRecursiveCte recursiveCte, C context) { - return visit(recursiveCte, context); + public R visitLogicalRecursiveUnion(LogicalRecursiveUnion recursiveUnion, + C context) { + return visit(recursiveUnion, context); + } + + public R visitLogicalRecursiveUnionAnchor(LogicalRecursiveUnionAnchor recursiveAnchor, + C context) { + return visit(recursiveAnchor, context); } - public R visitLogicalRecursiveCteRecursiveChild(LogicalRecursiveCteRecursiveChild recursiveChild, + public R visitLogicalRecursiveUnionProducer(LogicalRecursiveUnionProducer recursiveProducer, C context) { - return visit(recursiveChild, context); + return visit(recursiveProducer, context); } public R visitLogicalLimit(LogicalLimit limit, C context) { @@ -394,13 +402,19 @@ public R visitPhysicalUnion(PhysicalUnion union, C context) { return visitPhysicalSetOperation(union, context); } - public R visitPhysicalRecursiveCte(PhysicalRecursiveCte recursiveCte, C context) { - return visit(recursiveCte, context); + public R visitPhysicalRecursiveUnion(PhysicalRecursiveUnion recursiveUnion, + C context) { + return visit(recursiveUnion, context); + } + + public R visitPhysicalRecursiveUnionAnchor(PhysicalRecursiveUnionAnchor recursiveUnionAnchor, + C context) { + return visit(recursiveUnionAnchor, context); } - public R visitPhysicalRecursiveCteRecursiveChild(PhysicalRecursiveCteRecursiveChild recursiveChild, + public R visitPhysicalRecursiveUnionProducer(PhysicalRecursiveUnionProducer recursiveUnionProducer, C context) { - return visit(recursiveChild, context); + return visit(recursiveUnionProducer, context); } public R visitAbstractPhysicalSort(AbstractPhysicalSort sort, C 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 index b325849b1b1022..d9593ee12fe1c5 100644 --- 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 @@ -30,11 +30,11 @@ import org.apache.doris.nereids.trees.plans.logical.LogicalOdbcScan; 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.LogicalRecursiveCteScan; 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.logical.LogicalTestScan; +import org.apache.doris.nereids.trees.plans.logical.LogicalWorkTableReference; import org.apache.doris.nereids.trees.plans.physical.PhysicalCTEConsumer; import org.apache.doris.nereids.trees.plans.physical.PhysicalCatalogRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalDeferMaterializeOlapScan; @@ -46,10 +46,10 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalOdbcScan; 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.PhysicalRecursiveCteScan; 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; +import org.apache.doris.nereids.trees.plans.physical.PhysicalWorkTableReference; /** * relation visitor @@ -141,8 +141,8 @@ default R visitLogicalTestScan(LogicalTestScan testScan, C context) { return visitLogicalCatalogRelation(testScan, context); } - default R visitLogicalRecursiveCteScan(LogicalRecursiveCteScan recursiveCteScan, C context) { - return visitLogicalCatalogRelation(recursiveCteScan, context); + default R visitLogicalWorkTableReference(LogicalWorkTableReference logicalWorkTableReference, C context) { + return visitLogicalRelation(logicalWorkTableReference, context); } // ******************************* @@ -182,8 +182,8 @@ default R visitPhysicalDeferMaterializeOlapScan( return visitPhysicalCatalogRelation(deferMaterializeOlapScan, context); } - default R visitPhysicalRecursiveCteScan(PhysicalRecursiveCteScan recursiveCteScan, C context) { - return visitPhysicalCatalogRelation(recursiveCteScan, context); + default R visitPhysicalWorkTableReference(PhysicalWorkTableReference workTableReference, C context) { + return visitPhysicalRelation(workTableReference, context); } default R visitPhysicalOneRowRelation(PhysicalOneRowRelation oneRowRelation, C context) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java index b009aa626480d5..5411c030aeb5bb 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/DataType.java @@ -547,6 +547,10 @@ public boolean equals(Object o) { return true; } + public boolean equalsForRecursiveCte(Object o) { + return equals(o); + } + @Override public int hashCode() { return 0; diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructType.java index 8296ac282810e7..0c33a6d2decb1a 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/StructType.java @@ -116,6 +116,30 @@ public boolean equals(Object o) { return Objects.equals(fields, that.fields); } + @Override + public boolean equalsForRecursiveCte(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + if (!super.equals(o)) { + return false; + } + StructType other = (StructType) o; + if (fields.size() != other.fields.size()) { + return false; + } + for (int i = 0; i < fields.size(); ++i) { + if ((fields.get(i).isNullable() != other.fields.get(i).isNullable()) + || !fields.get(i).getDataType().equalsForRecursiveCte(other.fields.get(i).getDataType())) { + return false; + } + } + return true; + } + @Override public int hashCode() { return Objects.hash(super.hashCode(), fields); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VariantType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VariantType.java index 038fc0f472e538..2b2cb73d70ad33 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VariantType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/types/VariantType.java @@ -165,6 +165,27 @@ public boolean equals(Object o) { && Objects.equals(predefinedFields, other.predefinedFields); } + @Override + public boolean equalsForRecursiveCte(Object o) { + if (this == o) { + return true; + } + if (o == null || getClass() != o.getClass()) { + return false; + } + VariantType other = (VariantType) o; + if (predefinedFields.size() != other.predefinedFields.size()) { + return false; + } + for (int i = 0; i < predefinedFields.size(); ++i) { + if (!predefinedFields.get(i).getDataType() + .equalsForRecursiveCte(other.predefinedFields.get(i).getDataType())) { + return false; + } + } + return true; + } + @Override public int hashCode() { return Objects.hash(super.hashCode(), variantMaxSubcolumnsCount, enableTypedPathsToSparse, diff --git a/fe/fe-core/src/main/java/org/apache/doris/planner/RecursiveCteScanNode.java b/fe/fe-core/src/main/java/org/apache/doris/planner/RecursiveCteScanNode.java index e613aca7594904..6f01cf624fe432 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/planner/RecursiveCteScanNode.java +++ b/fe/fe-core/src/main/java/org/apache/doris/planner/RecursiveCteScanNode.java @@ -19,77 +19,21 @@ import org.apache.doris.analysis.Expr; import org.apache.doris.analysis.TupleDescriptor; -import org.apache.doris.catalog.Env; -import org.apache.doris.common.UserException; -import org.apache.doris.system.Backend; import org.apache.doris.thrift.TExplainLevel; -import org.apache.doris.thrift.TNetworkAddress; import org.apache.doris.thrift.TPlanNode; import org.apache.doris.thrift.TPlanNodeType; -import org.apache.doris.thrift.TScanRange; -import org.apache.doris.thrift.TScanRangeLocation; -import org.apache.doris.thrift.TScanRangeLocations; import com.google.common.base.MoreObjects; -import com.google.common.collect.Lists; - -import java.util.Collections; -import java.util.List; // Full scan of recursive cte temp table -public class RecursiveCteScanNode extends ScanNode { +public class RecursiveCteScanNode extends PlanNode { private final String recursiveCteName; public RecursiveCteScanNode(String recursiveCteName, PlanNodeId id, TupleDescriptor desc) { - super(id, desc, "RECURSIVE_CTE_SCAN"); + super(id, desc.getId().asList(), "RECURSIVE_CTE_SCAN"); this.recursiveCteName = recursiveCteName; } - public void initScanRangeLocations() throws UserException { - createScanRangeLocations(); - } - - @Override - protected void createScanRangeLocations() throws UserException { - scanRangeLocations = Lists.newArrayList(); - // randomly select 1 backend - List backendList = Lists.newArrayList(); - for (Backend be : Env.getCurrentSystemInfo().getBackendsByCurrentCluster().values()) { - if (be.isAlive()) { - backendList.add(be); - } - } - if (backendList.isEmpty()) { - throw new UserException("No Alive backends"); - } - Collections.shuffle(backendList); - Backend selectedBackend = backendList.get(0); - - // create scan range locations - TScanRangeLocation location = new TScanRangeLocation(); - location.setBackendId(selectedBackend.getId()); - location.setServer(new TNetworkAddress(selectedBackend.getHost(), selectedBackend.getBePort())); - TScanRangeLocations locations = new TScanRangeLocations(); - locations.addToLocations(location); - locations.setScanRange(new TScanRange()); - scanRangeLocations.add(locations); - } - - @Override - public List getScanRangeLocations(long maxScanRangeLength) { - return scanRangeLocations; - } - - @Override - public int getNumInstances() { - return 1; - } - - @Override - public int getScanRangeNum() { - return 1; - } - @Override public String getNodeExplainString(String prefix, TExplainLevel detailLevel) { StringBuilder output = new StringBuilder(); @@ -106,7 +50,12 @@ public String toString() { return MoreObjects.toStringHelper(this) .add("recursiveCteName", recursiveCteName) .add("id", getId().asInt()) - .add("tid", desc.getId().asInt()).toString(); + .add("tid", getTupleIds().get(0)).toString(); + } + + @Override + public int getNumInstances() { + return 1; } @Override diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java index 1fe3b3661c383b..d3d16748951c56 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java +++ b/fe/fe-core/src/main/java/org/apache/doris/qe/runtime/ThriftPlansBuilder.java @@ -597,67 +597,137 @@ private static void filterInstancesWhichReceiveDataFromRemote( private static Set setParamsForRecursiveCteNode(List distributedPlans, List runtimeFilters) { + /* + * Populate and attach recursive-CTE related Thrift structures used by + * backends (BE) to coordinate recursive Common Table Expression (CTE) + * execution across fragments and fragment instances. + * + * This method performs the following responsibilities: + * - Traverse the provided `distributedPlans` in bottom-up order (this + * ordering is expected by callers) and collect the set of network + * addresses (host + brpcPort) for every fragment. These addresses are + * used to reset and control recursive CTE child fragments from the + * producer side. + * - Detect `RecursiveCteScanNode` within fragments and build a + * `TRecCTETarget` for each such scan. A `TRecCTETarget` captures the + * network address and a representative fragment instance id and the + * scan node id that the recursive producer should send data to. + * Exactly one `RecursiveCteScanNode` is expected per fragment that + * contains a scan for a recursive CTE; otherwise an + * IllegalStateException is thrown. + * - For every `RecursiveCteNode` (producer/union node), collect its + * child fragments that implement the recursive side. For each child + * fragment, add the corresponding `TRecCTETarget` (if present) to + * the producer's target list and create `TRecCTEResetInfo` entries + * for all instances of that fragment. `TRecCTEResetInfo` entries + * carry the fragment id and addresses to be reset by the producer + * when a new recursion iteration begins. + * - Populate the `TRecCTENode` object attached to the + * `RecursiveCteNode`, including: whether it's `UNION ALL`, result + * expression lists (materialized result expressions converted to + * Thrift `TExpr`), list of targets, fragments-to-reset, runtime + * filter ids that must be reset on the recursive side, and a flag + * indicating whether this recursive CTE node is used by other + * recursive CTEs. + * + * How runtime filters are handled: + * - Build `runtimeFiltersToReset` by scanning provided + * `runtimeFilters`. A filter id is added if the filter has remote + * targets and if the recursive side (right child) contains the + * runtime-filter builder node. These filter ids are attached to the + * `TRecCTENode` so BE can reset the corresponding runtime filters + * between recursive iterations. + * + * Important assumptions and invariants: + * - `distributedPlans` must be ordered bottom-up so that child + * fragments (containing `RecursiveCteScanNode`) are visited before + * their producers. The implementation relies on this to pop + * consumed `TRecCTETarget` entries from `fragmentIdToRecCteTargetMap` + * to avoid a parent producer incorrectly picking up grandchild + * scan nodes. + * - Each fragment containing a `RecursiveCteScanNode` must have at + * least one assigned job (instance). If not, an + * IllegalStateException is thrown. + * - At most one `RecursiveCteScanNode` per fragment is supported; if + * more than one is found an IllegalStateException is thrown. + * + * @param distributedPlans ordered list of PipelineDistributedPlan in + * bottom-up traversal order + * @param runtimeFilters list of runtime filters to consider for reset + * @return set of fragment ids (as integers) that need to be notified + * to close for recursive CTE handling + */ + // fragments whose child recursive fragments need to be notified to close Set fragmentToNotifyClose = new HashSet<>(); + // mapping from fragment id -> TRecCTETarget (the scan node target info) Map fragmentIdToRecCteTargetMap = new TreeMap<>(); + // mapping from fragment id -> set of network addresses for all instances Map> fragmentIdToNetworkAddressMap = new TreeMap<>(); // distributedPlans is ordered in bottom up way, so does the fragments for (PipelineDistributedPlan plan : distributedPlans) { + // collect all assigned instance network addresses for this fragment List fragmentAssignedJobs = plan.getInstanceJobs(); Set networkAddresses = new TreeSet<>(); for (AssignedJob assignedJob : fragmentAssignedJobs) { DistributedPlanWorker distributedPlanWorker = assignedJob.getAssignedWorker(); + // use brpc port + host as the address used by BE for control/reset networkAddresses.add(new TNetworkAddress(distributedPlanWorker.host(), distributedPlanWorker.brpcPort())); } PlanFragment planFragment = plan.getFragmentJob().getFragment(); + // remember addresses for later when building reset infos fragmentIdToNetworkAddressMap.put(planFragment.getFragmentId(), networkAddresses); + // find RecursiveCteScanNode in this fragment (scan side of recursive CTE) List recursiveCteScanNodes = planFragment.getPlanRoot() .collectInCurrentFragment(RecursiveCteScanNode.class::isInstance); if (!recursiveCteScanNodes.isEmpty()) { + // validate there is exactly one scan node per fragment if (recursiveCteScanNodes.size() != 1) { throw new IllegalStateException( String.format("one fragment can only have 1 recursive cte scan node, but there is %d", recursiveCteScanNodes.size())); } + // scan fragments must have at least one assigned instance if (fragmentAssignedJobs.isEmpty()) { throw new IllegalStateException( "fragmentAssignedJobs is empty for recursive cte scan node"); } + // Build a TRecCTETarget using the first assigned instance as representative TRecCTETarget tRecCTETarget = new TRecCTETarget(); DistributedPlanWorker distributedPlanWorker = fragmentAssignedJobs.get(0).getAssignedWorker(); tRecCTETarget.setAddr(new TNetworkAddress(distributedPlanWorker.host(), distributedPlanWorker.brpcPort())); tRecCTETarget.setFragmentInstanceId(fragmentAssignedJobs.get(0).instanceId()); tRecCTETarget.setNodeId(recursiveCteScanNodes.get(0).getId().asInt()); - // find all RecursiveCteScanNode and its fragment id + // store the target for producers to reference later fragmentIdToRecCteTargetMap.put(planFragment.getFragmentId(), tRecCTETarget); } List recursiveCteNodes = planFragment.getPlanRoot() .collectInCurrentFragment(RecursiveCteNode.class::isInstance); for (RecursiveCteNode recursiveCteNode : recursiveCteNodes) { + // list of scan targets this producer should send recursive rows to List targets = new ArrayList<>(); + // reset infos for all instances of child fragments (used to reset state) List fragmentsToReset = new ArrayList<>(); - // recursiveCteNode's right child is recursive part (exchange node) - // so we get collect all child fragment from exchange node's child node + // The recursive side is under the right child; collect all fragments List childFragments = new ArrayList<>(); recursiveCteNode.getChild(1).getChild(0).getFragment().collectAll(PlanFragment.class::isInstance, childFragments); for (PlanFragment child : childFragments) { PlanFragmentId childFragmentId = child.getFragmentId(); - // the fragment need to be notified to close + // mark this child fragment id so it will be notified to close fragmentToNotifyClose.add(childFragmentId.asInt()); + // add target if a matching RecursiveCteScanNode was recorded TRecCTETarget tRecCTETarget = fragmentIdToRecCteTargetMap.getOrDefault(childFragmentId, null); if (tRecCTETarget != null) { - // one RecursiveCteNode can only have one corresponding RecursiveCteScanNode + // each producer can only map to one scan node target per child targets.add(tRecCTETarget); - // because we traverse the fragments in bottom-up way - // we can safely remove accessed RecursiveCteScanNode - // so the parent RecursiveCteNode won't see its grandson RecursiveCteScanNode - // but can only see its child RecursiveCteScanNode + // remove the entry so ancestor producers won't reuse a grandchild scan node fragmentIdToRecCteTargetMap.remove(childFragmentId); } + // get all instance addresses for this child fragment and build reset infos Set tNetworkAddresses = fragmentIdToNetworkAddressMap.get(childFragmentId); if (tNetworkAddresses == null) { throw new IllegalStateException( @@ -671,26 +741,34 @@ private static Set setParamsForRecursiveCteNode(List> materializedResultExprLists = recursiveCteNode.getMaterializedResultExprLists(); List> texprLists = new ArrayList<>(materializedResultExprLists.size()); for (List exprList : materializedResultExprLists) { texprLists.add(Expr.treesToThrift(exprList)); } // the recursive side's rf need to be reset + // determine which runtime filters on the recursive side must be reset List runtimeFiltersToReset = new ArrayList<>(runtimeFilters.size()); for (RuntimeFilter rf : runtimeFilters) { + // only consider filters that have remote targets and whose builder + // node is present in the recursive side (right child) if (rf.hasRemoteTargets() && recursiveCteNode.getChild(1).contains(node -> node == rf.getBuilderNode())) { runtimeFiltersToReset.add(rf.getFilterId().asInt()); } } // find recursiveCte used by other recursive cte + // detect whether this recursive CTE node is referenced by other + // recursive CTEs in the recursive side; needed to correctly + // indicate sharing/usage across recursive nodes Set recursiveCteNodesInRecursiveSide = new HashSet<>(); PlanNode rootPlan = distributedPlans.get(distributedPlans.size() - 1) .getFragmentJob().getFragment().getPlanRoot(); collectAllRecursiveCteNodesInRecursiveSide(rootPlan, false, recursiveCteNodesInRecursiveSide); boolean isUsedByOtherRecCte = recursiveCteNodesInRecursiveSide.contains(recursiveCteNode); + // build the Thrift TRecCTENode and attach it to the RecursiveCteNode TRecCTENode tRecCTENode = new TRecCTENode(); tRecCTENode.setIsUnionAll(recursiveCteNode.isUnionAll()); tRecCTENode.setTargets(targets); @@ -698,6 +776,7 @@ private static Set setParamsForRecursiveCteNode(List project.getProjects().get(0).child(0) instanceof Nullable), - logicalRecursiveCteRecursiveChild( + ).when(project -> project.getProjects().get(0).child(0) instanceof Nullable)), + logicalRecursiveUnionProducer( logicalProject( logicalProject( logicalCTEConsumer() @@ -313,12 +314,14 @@ public void testRecCteWithoutRecKeyword() { PlanChecker.from(connectContext) .analyze(sql) .matches( - logicalRecursiveCte( - logicalProject( - logicalOneRowRelation( + this.logicalRecursiveUnion( + logicalRecursiveUnionAnchor( + logicalProject( + logicalOneRowRelation( + ) ) ), - logicalRecursiveCteRecursiveChild( + logicalRecursiveUnionProducer( logicalProject( logicalProject( logicalJoin() @@ -348,10 +351,11 @@ public void testRecCteMultipleUnion() { ExplainCommand.ExplainLevel.ANALYZED_PLAN); MemoTestUtils.initMemoAndValidState(planner.getCascadesContext()); PlanChecker.from(planner.getCascadesContext()).matches( - logicalRecursiveCte( - logicalProject( - logicalUnion()), - logicalRecursiveCteRecursiveChild()).when(cte -> cte.getCteName().equals("xx"))); + this.logicalRecursiveUnion( + logicalRecursiveUnionAnchor( + logicalProject( + logicalUnion())), + logicalRecursiveUnionProducer()).when(cte -> cte.getCteName().equals("xx"))); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/CTEInlineTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/CTEInlineTest.java index 0a0efc8b5db3a6..e9767ef524bd75 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/CTEInlineTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/rules/rewrite/CTEInlineTest.java @@ -63,16 +63,16 @@ public void recCteInline() { ExplainCommand.ExplainLevel.REWRITTEN_PLAN); MemoTestUtils.initMemoAndValidState(planner.getCascadesContext()); PlanChecker.from(planner.getCascadesContext()).matches( - logicalRecursiveCte( + this.logicalRecursiveUnion( any( ), - logicalRecursiveCteRecursiveChild( + logicalRecursiveUnionProducer( logicalProject( logicalJoin( any(), logicalProject( logicalFilter( - logicalRecursiveCte().when(cte -> cte.getCteName().equals("xx")) + logicalRecursiveUnion().when(cte -> cte.getCteName().equals("xx")) ) ) ) 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 353f0c13863ae8..3be69d79de1ba9 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 @@ -352,7 +352,7 @@ public void pruneRecCte() { PlanChecker.from(planner.getCascadesContext()).matches( logicalProject( logicalFilter( - logicalRecursiveCte().when(cte -> cte.getOutput().size() == 3) + logicalRecursiveUnion().when(cte -> cte.getOutput().size() == 3) ) ).when(project -> project.getOutputs().size() == 1) ); diff --git a/regression-test/data/rec_cte_p0/rec_cte/rec_cte.out b/regression-test/data/rec_cte_p0/rec_cte/rec_cte.out index ba843a71ee5282..6789ed440045ec 100644 --- a/regression-test/data/rec_cte_p0/rec_cte/rec_cte.out +++ b/regression-test/data/rec_cte_p0/rec_cte/rec_cte.out @@ -951,3 +951,20 @@ 3 4 11 22 +-- !sql -- +1 1 +1 2 +1 3 +1 11 +3 1 +3 3 +3 4 +3 11 +11 1 +11 3 +11 11 +11 22 +22 1 +22 3 +22 11 + diff --git a/regression-test/suites/rec_cte_p0/rec_cte/rec_cte.groovy b/regression-test/suites/rec_cte_p0/rec_cte/rec_cte.groovy index d456f6ba998501..b5bc90544b462c 100644 --- a/regression-test/suites/rec_cte_p0/rec_cte/rec_cte.groovy +++ b/regression-test/suites/rec_cte_p0/rec_cte/rec_cte.groovy @@ -268,4 +268,83 @@ suite ("rec_cte") { ) SELECT * FROM t1 UNION select * from t2 ORDER BY 1,2; """ + + qt_sql """ + WITH RECURSIVE t1(k1, k2) AS ( + SELECT + 1, + 2 + UNION + SELECT + 3, + 4 + FROM + t1 + GROUP BY + k1 + ), + t2(k1, k2) AS ( + SELECT + 11, + 22 + UNION + SELECT + t2.k2, + tx.k1 + FROM + t1, + t2, + ( + WITH RECURSIVE t1(k1, k2) AS ( + SELECT + 1, + 2 + UNION + SELECT + 3, + 4 + FROM + t1 + GROUP BY + k1 + ), + t2(k1, k2) AS ( + SELECT + 11, + 22 + UNION + SELECT + t2.k1, + t2.k2 + FROM + t1, + t2 + ) + SELECT + * + FROM + t1 + UNION + select + * + from + t2 + ORDER BY + 1, + 2 + ) tx + ) + SELECT + * + FROM + t1 + UNION + select + * + from + t2 + ORDER BY + 1, + 2; + """ }