You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by mo...@apache.org on 2022/06/22 06:07:33 UTC

[doris] branch master updated: [Minor](nereids): Get plan directly (#10270)

This is an automated email from the ASF dual-hosted git repository.

morningman pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/doris.git


The following commit(s) were added to refs/heads/master by this push:
     new 239ef84835 [Minor](nereids): Get plan directly (#10270)
239ef84835 is described below

commit 239ef8483578c2cce43ec66cfd181cf36308e96f
Author: jakevin <30...@users.noreply.github.com>
AuthorDate: Wed Jun 22 14:07:28 2022 +0800

    [Minor](nereids): Get plan directly (#10270)
---
 .../java/org/apache/doris/nereids/Planner.java     | 39 +++++++++++---
 .../java/org/apache/doris/nereids/memo/Group.java  | 39 +++++++++++---
 .../java/org/apache/doris/nereids/memo/Memo.java   | 16 ++++--
 .../org/apache/doris/nereids/memo/MemoTest.java    | 60 ++++++++++++++++++++++
 .../apache/doris/nereids/plan/TestPlanOutput.java  |  4 +-
 5 files changed, 139 insertions(+), 19 deletions(-)

diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/Planner.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/Planner.java
index 48ae896a3d..ab9396b35e 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/Planner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/Planner.java
@@ -21,12 +21,18 @@ import org.apache.doris.common.AnalysisException;
 import org.apache.doris.nereids.jobs.cascades.OptimizeGroupJob;
 import org.apache.doris.nereids.jobs.rewrite.RewriteBottomUpJob;
 import org.apache.doris.nereids.memo.Group;
+import org.apache.doris.nereids.memo.GroupExpression;
 import org.apache.doris.nereids.memo.Memo;
 import org.apache.doris.nereids.properties.PhysicalProperties;
+import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan;
 import org.apache.doris.qe.ConnectContext;
 
+import com.google.common.collect.Lists;
+
+import java.util.List;
+
 /**
  * Planner to do query plan in Nereids.
  */
@@ -43,10 +49,8 @@ public class Planner {
      * @throws AnalysisException throw exception if failed in ant stage
      */
     // TODO: refactor, just demo code here
-    public PhysicalPlan plan(
-            LogicalPlan plan,
-            PhysicalProperties outputProperties,
-            ConnectContext connectContext) throws AnalysisException {
+    public PhysicalPlan plan(LogicalPlan plan, PhysicalProperties outputProperties, ConnectContext connectContext)
+            throws AnalysisException {
         Memo memo = new Memo();
         memo.initialize(plan);
 
@@ -58,14 +62,35 @@ public class Planner {
 
         plannerContext.getOptimizerContext().pushJob(new OptimizeGroupJob(getRoot(), plannerContext));
         plannerContext.getOptimizerContext().getJobScheduler().executeJobPool(plannerContext);
-        return getBestPlan();
+
+        // Get plan directly. Just for SSB.
+        return getRoot().extractPlan();
     }
 
     public Group getRoot() {
         return plannerContext.getOptimizerContext().getMemo().getRoot();
     }
 
-    private PhysicalPlan getBestPlan() {
-        return null;
+    private PhysicalPlan chooseBestPlan(Group rootGroup, PhysicalProperties physicalProperties)
+            throws AnalysisException {
+        GroupExpression groupExpression = rootGroup.getLowestCostPlan(physicalProperties).orElseThrow(
+                () -> new AnalysisException("lowestCostPlans with physicalProperties doesn't exist")).second;
+        List<PhysicalProperties> inputPropertiesList = groupExpression.getInputPropertiesList(physicalProperties);
+
+        List<Plan> planChildren = Lists.newArrayList();
+        for (int i = 0; i < groupExpression.arity(); i++) {
+            planChildren.add(chooseBestPlan(groupExpression.child(i), inputPropertiesList.get(i)));
+        }
+
+        Plan plan = ((PhysicalPlan) groupExpression.getOperator().toTreeNode(groupExpression)).withChildren(
+                planChildren);
+        if (!(plan instanceof PhysicalPlan)) {
+            throw new AnalysisException("generate logical plan");
+        }
+        PhysicalPlan physicalPlan = (PhysicalPlan) plan;
+
+        // TODO: set (logical and physical)properties/statistics/... for physicalPlan.
+
+        return physicalPlan;
     }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java
index ad77a424b7..929d8c86fe 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Group.java
@@ -18,13 +18,15 @@
 package org.apache.doris.nereids.memo;
 
 import org.apache.doris.common.Pair;
+import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.operators.plans.logical.LogicalOperator;
 import org.apache.doris.nereids.properties.LogicalProperties;
 import org.apache.doris.nereids.properties.PhysicalProperties;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan;
 
-import com.clearspring.analytics.util.Lists;
-import com.clearspring.analytics.util.Preconditions;
-import org.springframework.util.CollectionUtils;
+import com.google.common.base.Preconditions;
+import com.google.common.collect.Lists;
 
 import java.util.List;
 import java.util.Map;
@@ -104,8 +106,8 @@ public class Group {
      * @param newExpression new logical group expression
      * @return old logical group expression
      */
-    public GroupExpression rewriteLogicalExpression(
-            GroupExpression newExpression, LogicalProperties logicalProperties) {
+    public GroupExpression rewriteLogicalExpression(GroupExpression newExpression,
+            LogicalProperties logicalProperties) {
         newExpression.setParent(this);
         this.logicalProperties = logicalProperties;
         GroupExpression oldExpression = getLogicalExpression();
@@ -126,6 +128,10 @@ public class Group {
         return logicalExpressions;
     }
 
+    public GroupExpression logicalExpressionsAt(int index) {
+        return logicalExpressions.get(index);
+    }
+
     /**
      * Get the first logical group expression in this group.
      * If there is no logical group expression or more than one, throw an exception.
@@ -168,12 +174,33 @@ public class Group {
      * @return {@link Optional} of cost and {@link GroupExpression} of physical plan pair.
      */
     public Optional<Pair<Double, GroupExpression>> getLowestCostPlan(PhysicalProperties physicalProperties) {
-        if (physicalProperties == null || CollectionUtils.isEmpty(lowestCostPlans)) {
+        if (physicalProperties == null || lowestCostPlans.isEmpty()) {
             return Optional.empty();
         }
         return Optional.ofNullable(lowestCostPlans.get(physicalProperties));
     }
 
+    /**
+     * Get the first Plan from Memo.
+     */
+    public PhysicalPlan extractPlan() throws AnalysisException {
+        GroupExpression groupExpression = this.logicalExpressionsAt(0);
+
+        List<Plan> planChildren = com.google.common.collect.Lists.newArrayList();
+        for (int i = 0; i < groupExpression.arity(); i++) {
+            planChildren.add(groupExpression.child(i).extractPlan());
+        }
+
+        Plan plan = ((PhysicalPlan) groupExpression.getOperator().toTreeNode(groupExpression)).withChildren(
+                planChildren);
+        if (!(plan instanceof PhysicalPlan)) {
+            throw new AnalysisException("generate logical plan");
+        }
+        PhysicalPlan physicalPlan = (PhysicalPlan) plan;
+
+        return physicalPlan;
+    }
+
     @Override
     public boolean equals(Object o) {
         if (this == o) {
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
index e05eb640e6..761ac4c342 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/memo/Memo.java
@@ -48,8 +48,16 @@ public class Memo {
         return root;
     }
 
+    public List<Group> getGroups() {
+        return groups;
+    }
+
+    public Map<GroupExpression, GroupExpression> getGroupExpressions() {
+        return groupExpressions;
+    }
+
     /**
-     * Add node to Memo.
+     * Add plan to Memo.
      * TODO: add ut later
      *
      * @param node {@link Plan} or {@link Expression} to be added
@@ -105,7 +113,7 @@ public class Memo {
      * @return existing groupExpression in memo or newly generated groupExpression
      */
     private GroupExpression insertOrRewriteGroupExpression(GroupExpression groupExpression, Group target,
-                                                           boolean rewrite, LogicalProperties logicalProperties) {
+            boolean rewrite, LogicalProperties logicalProperties) {
         GroupExpression existedGroupExpression = groupExpressions.get(groupExpression);
         if (existedGroupExpression != null) {
             if (target != null && !target.getGroupId().equals(existedGroupExpression.getParent().getGroupId())) {
@@ -144,7 +152,7 @@ public class Memo {
             return;
         }
         List<GroupExpression> needReplaceChild = Lists.newArrayList();
-        for (GroupExpression groupExpression : groupExpressions.values()) {
+        groupExpressions.values().forEach(groupExpression -> {
             if (groupExpression.children().contains(source)) {
                 if (groupExpression.getParent().equals(destination)) {
                     // cycle, we should not merge
@@ -152,7 +160,7 @@ public class Memo {
                 }
                 needReplaceChild.add(groupExpression);
             }
-        }
+        });
         for (GroupExpression groupExpression : needReplaceChild) {
             groupExpressions.remove(groupExpression);
             List<Group> children = groupExpression.children();
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java
new file mode 100644
index 0000000000..40b8eeed1e
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/memo/MemoTest.java
@@ -0,0 +1,60 @@
+// 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.memo;
+
+import org.apache.doris.nereids.analyzer.UnboundRelation;
+import org.apache.doris.nereids.operators.OperatorType;
+import org.apache.doris.nereids.operators.plans.logical.LogicalProject;
+import org.apache.doris.nereids.trees.expressions.SlotReference;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.Plans;
+import org.apache.doris.nereids.types.StringType;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import org.junit.Assert;
+import org.junit.Test;
+
+public class MemoTest implements Plans {
+    @Test
+    public void testInitialize() {
+        UnboundRelation unboundRelation = new UnboundRelation(Lists.newArrayList("test"));
+        LogicalProject insideProject = new LogicalProject(
+                ImmutableList.of(new SlotReference("name", StringType.INSTANCE, true, ImmutableList.of("test"))));
+        LogicalProject rootProject = new LogicalProject(
+                ImmutableList.of(new SlotReference("name", StringType.INSTANCE, true, ImmutableList.of("test"))));
+
+        // Project -> Project -> Relation
+        Plan root = plan(rootProject, plan(insideProject, plan(unboundRelation)));
+
+        Memo memo = new Memo();
+        memo.initialize(root);
+
+        Group rootGroup = memo.getRoot();
+
+        Assert.assertEquals(3, memo.getGroups().size());
+        Assert.assertEquals(3, memo.getGroupExpressions().size());
+
+        Assert.assertEquals(OperatorType.LOGICAL_PROJECT, rootGroup.logicalExpressionsAt(0).getOperator().getType());
+        Assert.assertEquals(OperatorType.LOGICAL_PROJECT,
+                rootGroup.logicalExpressionsAt(0).child(0).logicalExpressionsAt(0).getOperator().getType());
+        Assert.assertEquals(OperatorType.LOGICAL_UNBOUND_RELATION,
+                rootGroup.logicalExpressionsAt(0).child(0).logicalExpressionsAt(0).child(0).logicalExpressionsAt(0)
+                        .getOperator().getType());
+    }
+}
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/plan/TestPlanOutput.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/plan/TestPlanOutput.java
index 5f037de5e6..ec0df43660 100644
--- a/fe/fe-core/src/test/java/org/apache/doris/nereids/plan/TestPlanOutput.java
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/plan/TestPlanOutput.java
@@ -50,7 +50,7 @@ public class TestPlanOutput implements Plans {
             new LogicalRelation(table, ImmutableList.of("a"))
         );
         List<Slot> output = relationPlan.getOutput();
-        Assertions.assertTrue(output.size() == 2);
+        Assertions.assertEquals(2, output.size());
         Assertions.assertEquals(output.get(0).getName(), "id");
         Assertions.assertEquals(output.get(0).getQualifiedName(), "a.id");
         Assertions.assertEquals(output.get(0).getDataType(), IntegerType.INSTANCE);
@@ -90,7 +90,7 @@ public class TestPlanOutput implements Plans {
         // column prune
         LogicalLeafPlan<LogicalRelation> newPlan = relationPlan.withOutput(ImmutableList.of(output.get(0)));
         output = newPlan.getOutput();
-        Assertions.assertTrue(output.size() == 1);
+        Assertions.assertEquals(1, output.size());
         Assertions.assertEquals(output.get(0).getName(), "id");
         Assertions.assertEquals(output.get(0).getQualifiedName(), "a.id");
         Assertions.assertEquals(output.get(0).getDataType(), IntegerType.INSTANCE);


---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org