You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2022/06/21 08:49:48 UTC

[GitHub] [doris] Kikyou1997 opened a new pull request, #10304: [feature](nereids) Integrate nereids into current SQL process framework

Kikyou1997 opened a new pull request, #10304:
URL: https://github.com/apache/doris/pull/10304

   # Proposed changes
   
   Integrate nereids into current SQL process framework.
   
   ## Problem Summary:
   
   1. Add PlannerAdapter and LogicalPlanner
   2. Add some branch to the ConnectProcessor and StmtExecutor
   3. Add support for  multiple statement parsing in the Parser
   
   ## Checklist(Required)
   
   1. Does it affect the original behavior: (Yes)
   4. Has unit tests been added: (No)
   5. Has document been added or modified: (No Need)
   6. Does it need to update dependencies: (No)
   7. Are there any changes that cannot be rolled back: (No)
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] yinzhijian commented on a diff in pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
yinzhijian commented on code in PR #10304:
URL: https://github.com/apache/doris/pull/10304#discussion_r904463311


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/PlannerAdapter.java:
##########
@@ -0,0 +1,75 @@
+// 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;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.StatementBase;
+import org.apache.doris.common.UserException;
+import org.apache.doris.nereids.properties.PhysicalProperties;
+import org.apache.doris.nereids.trees.plans.PhysicalPlanTranslator;
+import org.apache.doris.nereids.trees.plans.PlanContext;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlanAdapter;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan;
+import org.apache.doris.planner.PlanFragment;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.planner.ScanNode;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.thrift.TQueryOptions;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * TODO: Abstract a common interface for both nereids and stale.
+ *       optimizer may be better.
+ * This class is used for the compatibility and code reuse in.
+ * @see org.apache.doris.qe.StmtExecutor
+ */
+public class PlannerAdapter extends Planner {
+
+    private final org.apache.doris.nereids.Planner planner;
+    private final ConnectContext ctx;
+    private List<ScanNode> scanNodeList = null;
+
+    public PlannerAdapter(ConnectContext ctx) {
+        this.planner = new org.apache.doris.nereids.Planner();
+        this.ctx = ctx;
+    }
+
+    @Override
+    public void plan(StatementBase queryStmt, Analyzer analyzer, TQueryOptions queryOptions) throws UserException {
+        if (!(queryStmt instanceof LogicalPlanAdapter)) {
+            throw new RuntimeException("Wrong type of queryStmt, expected: <? extends LogicalPlanAdapter>");
+        }
+        LogicalPlanAdapter logicalPlanAdapter = (LogicalPlanAdapter) queryStmt;
+        PhysicalPlan physicalPlan = planner.plan(logicalPlanAdapter.getLogicalPlan(), new PhysicalProperties(), ctx);
+        PhysicalPlanTranslator physicalPlanTranslator = new PhysicalPlanTranslator();
+        PlanContext planContext = new PlanContext();
+        physicalPlanTranslator.translatePlan(physicalPlan, planContext);
+        fragments = new ArrayList<>(planContext.getPlanFragmentList());
+        PlanFragment root = fragments.get(fragments.size() - 1);
+        root.getPlanRoot().convertToVectoriezd();

Review Comment:
   You may need to add ```if (VectorizedUtil.isVectorized()) ``` here



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] Kikyou1997 commented on a diff in pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
Kikyou1997 commented on code in PR #10304:
URL: https://github.com/apache/doris/pull/10304#discussion_r903713020


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/PlannerAdapter.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;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.StatementBase;
+import org.apache.doris.common.UserException;
+import org.apache.doris.nereids.properties.PhysicalProperties;
+import org.apache.doris.nereids.trees.plans.PhysicalPlanTranslator;
+import org.apache.doris.nereids.trees.plans.PlanContext;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlanAdapter;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.thrift.TQueryOptions;
+
+import java.util.ArrayList;
+
+/**
+ * This class is used for the compatibility and code reuse in.
+ * @see org.apache.doris.qe.StmtExecutor
+ */
+public class PlannerAdapter extends Planner {

Review Comment:
   Okay, I'll refactor it in the future



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] EmmyMiao87 commented on a diff in pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on code in PR #10304:
URL: https://github.com/apache/doris/pull/10304#discussion_r905781337


##########
fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java:
##########
@@ -6,430 +6,47 @@
 // "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
+//  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.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/Planner.java
-// and modified by Doris
 
 package org.apache.doris.planner;
 
-import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.ExplainOptions;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.InsertStmt;
-import org.apache.doris.analysis.QueryStmt;
-import org.apache.doris.analysis.SelectStmt;
-import org.apache.doris.analysis.SlotDescriptor;
-import org.apache.doris.analysis.SlotId;
 import org.apache.doris.analysis.StatementBase;
-import org.apache.doris.analysis.StorageBackend;
-import org.apache.doris.analysis.TupleDescriptor;
-import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.common.UserException;
-import org.apache.doris.common.profile.PlanTreeBuilder;
-import org.apache.doris.common.profile.PlanTreePrinter;
-import org.apache.doris.common.util.VectorizedUtil;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException;
-import org.apache.doris.thrift.TExplainLevel;
-import org.apache.doris.thrift.TQueryOptions;
-import org.apache.doris.thrift.TRuntimeFilterMode;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-/**
- * The planner is responsible for turning parse trees into plan fragments that can be shipped off to backends for
- * execution.
- */
-public class Planner {
-    private static final Logger LOG = LogManager.getLogger(Planner.class);
-
-    private boolean isBlockQuery = false;
+public abstract class Planner {
 
     protected ArrayList<PlanFragment> fragments = Lists.newArrayList();
 
-    private PlannerContext plannerContext;
-    private SingleNodePlanner singleNodePlanner;
-    private DistributedPlanner distributedPlanner;
+    protected boolean isBlockQuery = false;
 
-    public boolean isBlockQuery() {
-        return isBlockQuery;
-    }
+    public abstract List<ScanNode> getScanNodes();
 
-    public List<PlanFragment> getFragments() {
-        return fragments;
-    }
-
-    public PlannerContext getPlannerContext() {
-        return plannerContext;
-    }
-
-    public List<ScanNode> getScanNodes() {
-        if (singleNodePlanner == null) {
-            return Lists.newArrayList();
-        }
-        return singleNodePlanner.getScanNodes();
-    }
-
-    public void plan(StatementBase queryStmt, Analyzer analyzer, TQueryOptions queryOptions)
-            throws UserException {
-        createPlanFragments(queryStmt, analyzer, queryOptions);
-    }
+    public abstract void plan(StatementBase queryStmt,
+            org.apache.doris.thrift.TQueryOptions queryOptions) throws UserException;
 
-    /**
-     */
-    private void setResultExprScale(Analyzer analyzer, ArrayList<Expr> outputExprs) {
-        for (TupleDescriptor tupleDesc : analyzer.getDescTbl().getTupleDescs()) {
-            for (SlotDescriptor slotDesc : tupleDesc.getSlots()) {
-                for (Expr expr : outputExprs) {
-                    List<SlotId> slotList = Lists.newArrayList();
-                    expr.getIds(null, slotList);
-                    if (PrimitiveType.DECIMALV2 != expr.getType().getPrimitiveType()) {
-                        continue;
-                    }
-
-                    if (PrimitiveType.DECIMALV2 != slotDesc.getType().getPrimitiveType()) {
-                        continue;
-                    }
-
-                    if (slotList.contains(slotDesc.getId()) && null != slotDesc.getColumn()) {
-                        int outputScale = slotDesc.getColumn().getScale();
-                        if (outputScale >= 0) {
-                            if (outputScale > expr.getOutputScale()) {
-                                expr.setOutputScale(outputScale);
-                            }
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Return combined explain string for all plan fragments.
-     */
     public String getExplainString(List<PlanFragment> fragments, ExplainOptions explainOptions) {

Review Comment:
   ```suggestion
       public String getExplainString(ExplainOptions explainOptions) {
           // use fragments directly
       }
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java:
##########
@@ -6,430 +6,47 @@
 // "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
+//  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.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/Planner.java
-// and modified by Doris
 
 package org.apache.doris.planner;
 
-import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.ExplainOptions;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.InsertStmt;
-import org.apache.doris.analysis.QueryStmt;
-import org.apache.doris.analysis.SelectStmt;
-import org.apache.doris.analysis.SlotDescriptor;
-import org.apache.doris.analysis.SlotId;
 import org.apache.doris.analysis.StatementBase;
-import org.apache.doris.analysis.StorageBackend;
-import org.apache.doris.analysis.TupleDescriptor;
-import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.common.UserException;
-import org.apache.doris.common.profile.PlanTreeBuilder;
-import org.apache.doris.common.profile.PlanTreePrinter;
-import org.apache.doris.common.util.VectorizedUtil;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException;
-import org.apache.doris.thrift.TExplainLevel;
-import org.apache.doris.thrift.TQueryOptions;
-import org.apache.doris.thrift.TRuntimeFilterMode;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-/**
- * The planner is responsible for turning parse trees into plan fragments that can be shipped off to backends for
- * execution.
- */
-public class Planner {
-    private static final Logger LOG = LogManager.getLogger(Planner.class);
-
-    private boolean isBlockQuery = false;
+public abstract class Planner {
 
     protected ArrayList<PlanFragment> fragments = Lists.newArrayList();
 
-    private PlannerContext plannerContext;
-    private SingleNodePlanner singleNodePlanner;
-    private DistributedPlanner distributedPlanner;
+    protected boolean isBlockQuery = false;
 
-    public boolean isBlockQuery() {
-        return isBlockQuery;
-    }
+    public abstract List<ScanNode> getScanNodes();
 
-    public List<PlanFragment> getFragments() {
-        return fragments;
-    }
-
-    public PlannerContext getPlannerContext() {
-        return plannerContext;
-    }
-
-    public List<ScanNode> getScanNodes() {
-        if (singleNodePlanner == null) {
-            return Lists.newArrayList();
-        }
-        return singleNodePlanner.getScanNodes();
-    }
-
-    public void plan(StatementBase queryStmt, Analyzer analyzer, TQueryOptions queryOptions)
-            throws UserException {
-        createPlanFragments(queryStmt, analyzer, queryOptions);
-    }
+    public abstract void plan(StatementBase queryStmt,
+            org.apache.doris.thrift.TQueryOptions queryOptions) throws UserException;
 
-    /**
-     */
-    private void setResultExprScale(Analyzer analyzer, ArrayList<Expr> outputExprs) {
-        for (TupleDescriptor tupleDesc : analyzer.getDescTbl().getTupleDescs()) {
-            for (SlotDescriptor slotDesc : tupleDesc.getSlots()) {
-                for (Expr expr : outputExprs) {
-                    List<SlotId> slotList = Lists.newArrayList();
-                    expr.getIds(null, slotList);
-                    if (PrimitiveType.DECIMALV2 != expr.getType().getPrimitiveType()) {
-                        continue;
-                    }
-
-                    if (PrimitiveType.DECIMALV2 != slotDesc.getType().getPrimitiveType()) {
-                        continue;
-                    }
-
-                    if (slotList.contains(slotDesc.getId()) && null != slotDesc.getColumn()) {
-                        int outputScale = slotDesc.getColumn().getScale();
-                        if (outputScale >= 0) {
-                            if (outputScale > expr.getOutputScale()) {
-                                expr.setOutputScale(outputScale);
-                            }
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Return combined explain string for all plan fragments.
-     */
     public String getExplainString(List<PlanFragment> fragments, ExplainOptions explainOptions) {
-        Preconditions.checkNotNull(explainOptions);
-        if (explainOptions.isGraph()) {
-            // print the plan graph
-            PlanTreeBuilder builder = new PlanTreeBuilder(fragments);
-            try {
-                builder.build();
-            } catch (UserException e) {
-                LOG.warn("Failed to build explain plan tree", e);
-                return e.getMessage();
-            }
-            return PlanTreePrinter.printPlanExplanation(builder.getTreeRoot());
-        }
-
-        // print text plan
-        TExplainLevel explainLevel = explainOptions.isVerbose() ? TExplainLevel.VERBOSE : TExplainLevel.NORMAL;
-        StringBuilder str = new StringBuilder();
-        for (int i = 0; i < fragments.size(); ++i) {
-            PlanFragment fragment = fragments.get(i);
-            if (i > 0) {
-                // a blank line between plan fragments
-                str.append("\n");
-            }
-            str.append("PLAN FRAGMENT " + i + "\n");
-            str.append(fragment.getExplainString(explainLevel));
-        }
-        if (explainLevel == TExplainLevel.VERBOSE) {
-            str.append(plannerContext.getRootAnalyzer().getDescTbl().getExplainString());
-        }
-        return str.toString();
+        return "Not implemented yet";

Review Comment:
   Since the `getExplainString` function only needs to use the fragments variable, the `getExplainString` implementation of the old and new optimizers can be unified in the abstract class



##########
fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java:
##########
@@ -6,430 +6,47 @@
 // "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
+//  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.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/Planner.java
-// and modified by Doris
 
 package org.apache.doris.planner;
 
-import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.ExplainOptions;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.InsertStmt;
-import org.apache.doris.analysis.QueryStmt;
-import org.apache.doris.analysis.SelectStmt;
-import org.apache.doris.analysis.SlotDescriptor;
-import org.apache.doris.analysis.SlotId;
 import org.apache.doris.analysis.StatementBase;
-import org.apache.doris.analysis.StorageBackend;
-import org.apache.doris.analysis.TupleDescriptor;
-import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.common.UserException;
-import org.apache.doris.common.profile.PlanTreeBuilder;
-import org.apache.doris.common.profile.PlanTreePrinter;
-import org.apache.doris.common.util.VectorizedUtil;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException;
-import org.apache.doris.thrift.TExplainLevel;
-import org.apache.doris.thrift.TQueryOptions;
-import org.apache.doris.thrift.TRuntimeFilterMode;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-/**
- * The planner is responsible for turning parse trees into plan fragments that can be shipped off to backends for
- * execution.
- */
-public class Planner {
-    private static final Logger LOG = LogManager.getLogger(Planner.class);
-
-    private boolean isBlockQuery = false;
+public abstract class Planner {
 
     protected ArrayList<PlanFragment> fragments = Lists.newArrayList();
 
-    private PlannerContext plannerContext;
-    private SingleNodePlanner singleNodePlanner;
-    private DistributedPlanner distributedPlanner;
+    protected boolean isBlockQuery = false;
 
-    public boolean isBlockQuery() {
-        return isBlockQuery;
-    }
+    public abstract List<ScanNode> getScanNodes();
 
-    public List<PlanFragment> getFragments() {
-        return fragments;
-    }
-
-    public PlannerContext getPlannerContext() {
-        return plannerContext;
-    }
-
-    public List<ScanNode> getScanNodes() {
-        if (singleNodePlanner == null) {
-            return Lists.newArrayList();
-        }
-        return singleNodePlanner.getScanNodes();
-    }
-
-    public void plan(StatementBase queryStmt, Analyzer analyzer, TQueryOptions queryOptions)
-            throws UserException {
-        createPlanFragments(queryStmt, analyzer, queryOptions);
-    }
+    public abstract void plan(StatementBase queryStmt,
+            org.apache.doris.thrift.TQueryOptions queryOptions) throws UserException;

Review Comment:
   ```suggestion
               TQueryOptions queryOptions) throws UserException;
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] Kikyou1997 commented on a diff in pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
Kikyou1997 commented on code in PR #10304:
URL: https://github.com/apache/doris/pull/10304#discussion_r905819114


##########
fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java:
##########
@@ -6,430 +6,47 @@
 // "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
+//  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.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/Planner.java
-// and modified by Doris
 
 package org.apache.doris.planner;
 
-import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.ExplainOptions;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.InsertStmt;
-import org.apache.doris.analysis.QueryStmt;
-import org.apache.doris.analysis.SelectStmt;
-import org.apache.doris.analysis.SlotDescriptor;
-import org.apache.doris.analysis.SlotId;
 import org.apache.doris.analysis.StatementBase;
-import org.apache.doris.analysis.StorageBackend;
-import org.apache.doris.analysis.TupleDescriptor;
-import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.common.UserException;
-import org.apache.doris.common.profile.PlanTreeBuilder;
-import org.apache.doris.common.profile.PlanTreePrinter;
-import org.apache.doris.common.util.VectorizedUtil;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException;
-import org.apache.doris.thrift.TExplainLevel;
-import org.apache.doris.thrift.TQueryOptions;
-import org.apache.doris.thrift.TRuntimeFilterMode;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-/**
- * The planner is responsible for turning parse trees into plan fragments that can be shipped off to backends for
- * execution.
- */
-public class Planner {
-    private static final Logger LOG = LogManager.getLogger(Planner.class);
-
-    private boolean isBlockQuery = false;
+public abstract class Planner {
 
     protected ArrayList<PlanFragment> fragments = Lists.newArrayList();
 
-    private PlannerContext plannerContext;
-    private SingleNodePlanner singleNodePlanner;
-    private DistributedPlanner distributedPlanner;
+    protected boolean isBlockQuery = false;
 
-    public boolean isBlockQuery() {
-        return isBlockQuery;
-    }
+    public abstract List<ScanNode> getScanNodes();
 
-    public List<PlanFragment> getFragments() {
-        return fragments;
-    }
-
-    public PlannerContext getPlannerContext() {
-        return plannerContext;
-    }
-
-    public List<ScanNode> getScanNodes() {
-        if (singleNodePlanner == null) {
-            return Lists.newArrayList();
-        }
-        return singleNodePlanner.getScanNodes();
-    }
-
-    public void plan(StatementBase queryStmt, Analyzer analyzer, TQueryOptions queryOptions)
-            throws UserException {
-        createPlanFragments(queryStmt, analyzer, queryOptions);
-    }
+    public abstract void plan(StatementBase queryStmt,
+            org.apache.doris.thrift.TQueryOptions queryOptions) throws UserException;

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10304:
URL: https://github.com/apache/doris/pull/10304#issuecomment-1165425006

   PR approved by at least one committer and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] EmmyMiao87 merged pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 merged PR #10304:
URL: https://github.com/apache/doris/pull/10304


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] github-actions[bot] commented on pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
github-actions[bot] commented on PR #10304:
URL: https://github.com/apache/doris/pull/10304#issuecomment-1162597637

   PR approved by anyone and no changes requested.


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] EmmyMiao87 commented on a diff in pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on code in PR #10304:
URL: https://github.com/apache/doris/pull/10304#discussion_r904481612


##########
fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java:
##########
@@ -735,8 +737,12 @@ private void analyzeAndGenerateQueryPlan(TQueryOptions tQueryOptions) throws Use
         }
         plannerProfile.setQueryAnalysisFinishTime();
 
-        // create plan
-        planner = new Planner();
+        if (parsedStmt instanceof LogicalPlanAdapter) {

Review Comment:
   ```
   if (parsedStmt instanceof LogicalPlanAdapter) {
       planner = NereidsPlanner();
   } else {
       planner = OriginPlanner();
   }
   ```



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] Kikyou1997 commented on a diff in pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
Kikyou1997 commented on code in PR #10304:
URL: https://github.com/apache/doris/pull/10304#discussion_r905818983


##########
fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java:
##########
@@ -6,430 +6,47 @@
 // "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
+//  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.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/Planner.java
-// and modified by Doris
 
 package org.apache.doris.planner;
 
-import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.ExplainOptions;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.InsertStmt;
-import org.apache.doris.analysis.QueryStmt;
-import org.apache.doris.analysis.SelectStmt;
-import org.apache.doris.analysis.SlotDescriptor;
-import org.apache.doris.analysis.SlotId;
 import org.apache.doris.analysis.StatementBase;
-import org.apache.doris.analysis.StorageBackend;
-import org.apache.doris.analysis.TupleDescriptor;
-import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.common.UserException;
-import org.apache.doris.common.profile.PlanTreeBuilder;
-import org.apache.doris.common.profile.PlanTreePrinter;
-import org.apache.doris.common.util.VectorizedUtil;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException;
-import org.apache.doris.thrift.TExplainLevel;
-import org.apache.doris.thrift.TQueryOptions;
-import org.apache.doris.thrift.TRuntimeFilterMode;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-/**
- * The planner is responsible for turning parse trees into plan fragments that can be shipped off to backends for
- * execution.
- */
-public class Planner {
-    private static final Logger LOG = LogManager.getLogger(Planner.class);
-
-    private boolean isBlockQuery = false;
+public abstract class Planner {
 
     protected ArrayList<PlanFragment> fragments = Lists.newArrayList();
 
-    private PlannerContext plannerContext;
-    private SingleNodePlanner singleNodePlanner;
-    private DistributedPlanner distributedPlanner;
+    protected boolean isBlockQuery = false;
 
-    public boolean isBlockQuery() {
-        return isBlockQuery;
-    }
+    public abstract List<ScanNode> getScanNodes();
 
-    public List<PlanFragment> getFragments() {
-        return fragments;
-    }
-
-    public PlannerContext getPlannerContext() {
-        return plannerContext;
-    }
-
-    public List<ScanNode> getScanNodes() {
-        if (singleNodePlanner == null) {
-            return Lists.newArrayList();
-        }
-        return singleNodePlanner.getScanNodes();
-    }
-
-    public void plan(StatementBase queryStmt, Analyzer analyzer, TQueryOptions queryOptions)
-            throws UserException {
-        createPlanFragments(queryStmt, analyzer, queryOptions);
-    }
+    public abstract void plan(StatementBase queryStmt,
+            org.apache.doris.thrift.TQueryOptions queryOptions) throws UserException;
 
-    /**
-     */
-    private void setResultExprScale(Analyzer analyzer, ArrayList<Expr> outputExprs) {
-        for (TupleDescriptor tupleDesc : analyzer.getDescTbl().getTupleDescs()) {
-            for (SlotDescriptor slotDesc : tupleDesc.getSlots()) {
-                for (Expr expr : outputExprs) {
-                    List<SlotId> slotList = Lists.newArrayList();
-                    expr.getIds(null, slotList);
-                    if (PrimitiveType.DECIMALV2 != expr.getType().getPrimitiveType()) {
-                        continue;
-                    }
-
-                    if (PrimitiveType.DECIMALV2 != slotDesc.getType().getPrimitiveType()) {
-                        continue;
-                    }
-
-                    if (slotList.contains(slotDesc.getId()) && null != slotDesc.getColumn()) {
-                        int outputScale = slotDesc.getColumn().getScale();
-                        if (outputScale >= 0) {
-                            if (outputScale > expr.getOutputScale()) {
-                                expr.setOutputScale(outputScale);
-                            }
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Return combined explain string for all plan fragments.
-     */
     public String getExplainString(List<PlanFragment> fragments, ExplainOptions explainOptions) {
-        Preconditions.checkNotNull(explainOptions);
-        if (explainOptions.isGraph()) {
-            // print the plan graph
-            PlanTreeBuilder builder = new PlanTreeBuilder(fragments);
-            try {
-                builder.build();
-            } catch (UserException e) {
-                LOG.warn("Failed to build explain plan tree", e);
-                return e.getMessage();
-            }
-            return PlanTreePrinter.printPlanExplanation(builder.getTreeRoot());
-        }
-
-        // print text plan
-        TExplainLevel explainLevel = explainOptions.isVerbose() ? TExplainLevel.VERBOSE : TExplainLevel.NORMAL;
-        StringBuilder str = new StringBuilder();
-        for (int i = 0; i < fragments.size(); ++i) {
-            PlanFragment fragment = fragments.get(i);
-            if (i > 0) {
-                // a blank line between plan fragments
-                str.append("\n");
-            }
-            str.append("PLAN FRAGMENT " + i + "\n");
-            str.append(fragment.getExplainString(explainLevel));
-        }
-        if (explainLevel == TExplainLevel.VERBOSE) {
-            str.append(plannerContext.getRootAnalyzer().getDescTbl().getExplainString());
-        }
-        return str.toString();
+        return "Not implemented yet";

Review Comment:
   done



##########
fe/fe-core/src/main/java/org/apache/doris/planner/Planner.java:
##########
@@ -6,430 +6,47 @@
 // "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
+//  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.
-// This file is copied from
-// https://github.com/apache/impala/blob/branch-2.9.0/fe/src/main/java/org/apache/impala/Planner.java
-// and modified by Doris
 
 package org.apache.doris.planner;
 
-import org.apache.doris.analysis.Analyzer;
 import org.apache.doris.analysis.ExplainOptions;
-import org.apache.doris.analysis.Expr;
-import org.apache.doris.analysis.InsertStmt;
-import org.apache.doris.analysis.QueryStmt;
-import org.apache.doris.analysis.SelectStmt;
-import org.apache.doris.analysis.SlotDescriptor;
-import org.apache.doris.analysis.SlotId;
 import org.apache.doris.analysis.StatementBase;
-import org.apache.doris.analysis.StorageBackend;
-import org.apache.doris.analysis.TupleDescriptor;
-import org.apache.doris.catalog.PrimitiveType;
-import org.apache.doris.catalog.ScalarType;
 import org.apache.doris.common.UserException;
-import org.apache.doris.common.profile.PlanTreeBuilder;
-import org.apache.doris.common.profile.PlanTreePrinter;
-import org.apache.doris.common.util.VectorizedUtil;
-import org.apache.doris.qe.ConnectContext;
-import org.apache.doris.rewrite.mvrewrite.MVSelectFailedException;
-import org.apache.doris.thrift.TExplainLevel;
-import org.apache.doris.thrift.TQueryOptions;
-import org.apache.doris.thrift.TRuntimeFilterMode;
 
-import com.google.common.base.Preconditions;
 import com.google.common.collect.Lists;
-import org.apache.logging.log4j.LogManager;
-import org.apache.logging.log4j.Logger;
 
 import java.util.ArrayList;
-import java.util.Collections;
 import java.util.List;
 
-/**
- * The planner is responsible for turning parse trees into plan fragments that can be shipped off to backends for
- * execution.
- */
-public class Planner {
-    private static final Logger LOG = LogManager.getLogger(Planner.class);
-
-    private boolean isBlockQuery = false;
+public abstract class Planner {
 
     protected ArrayList<PlanFragment> fragments = Lists.newArrayList();
 
-    private PlannerContext plannerContext;
-    private SingleNodePlanner singleNodePlanner;
-    private DistributedPlanner distributedPlanner;
+    protected boolean isBlockQuery = false;
 
-    public boolean isBlockQuery() {
-        return isBlockQuery;
-    }
+    public abstract List<ScanNode> getScanNodes();
 
-    public List<PlanFragment> getFragments() {
-        return fragments;
-    }
-
-    public PlannerContext getPlannerContext() {
-        return plannerContext;
-    }
-
-    public List<ScanNode> getScanNodes() {
-        if (singleNodePlanner == null) {
-            return Lists.newArrayList();
-        }
-        return singleNodePlanner.getScanNodes();
-    }
-
-    public void plan(StatementBase queryStmt, Analyzer analyzer, TQueryOptions queryOptions)
-            throws UserException {
-        createPlanFragments(queryStmt, analyzer, queryOptions);
-    }
+    public abstract void plan(StatementBase queryStmt,
+            org.apache.doris.thrift.TQueryOptions queryOptions) throws UserException;
 
-    /**
-     */
-    private void setResultExprScale(Analyzer analyzer, ArrayList<Expr> outputExprs) {
-        for (TupleDescriptor tupleDesc : analyzer.getDescTbl().getTupleDescs()) {
-            for (SlotDescriptor slotDesc : tupleDesc.getSlots()) {
-                for (Expr expr : outputExprs) {
-                    List<SlotId> slotList = Lists.newArrayList();
-                    expr.getIds(null, slotList);
-                    if (PrimitiveType.DECIMALV2 != expr.getType().getPrimitiveType()) {
-                        continue;
-                    }
-
-                    if (PrimitiveType.DECIMALV2 != slotDesc.getType().getPrimitiveType()) {
-                        continue;
-                    }
-
-                    if (slotList.contains(slotDesc.getId()) && null != slotDesc.getColumn()) {
-                        int outputScale = slotDesc.getColumn().getScale();
-                        if (outputScale >= 0) {
-                            if (outputScale > expr.getOutputScale()) {
-                                expr.setOutputScale(outputScale);
-                            }
-                        }
-                    }
-                }
-            }
-        }
-    }
-
-    /**
-     * Return combined explain string for all plan fragments.
-     */
     public String getExplainString(List<PlanFragment> fragments, ExplainOptions explainOptions) {

Review Comment:
   done



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] EmmyMiao87 commented on a diff in pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
EmmyMiao87 commented on code in PR #10304:
URL: https://github.com/apache/doris/pull/10304#discussion_r903679070


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/PlannerAdapter.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;
+
+import org.apache.doris.analysis.Analyzer;
+import org.apache.doris.analysis.StatementBase;
+import org.apache.doris.common.UserException;
+import org.apache.doris.nereids.properties.PhysicalProperties;
+import org.apache.doris.nereids.trees.plans.PhysicalPlanTranslator;
+import org.apache.doris.nereids.trees.plans.PlanContext;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlanAdapter;
+import org.apache.doris.nereids.trees.plans.physical.PhysicalPlan;
+import org.apache.doris.planner.Planner;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.thrift.TQueryOptions;
+
+import java.util.ArrayList;
+
+/**
+ * This class is used for the compatibility and code reuse in.
+ * @see org.apache.doris.qe.StmtExecutor
+ */
+public class PlannerAdapter extends Planner {

Review Comment:
   Whether to abstract the original planner out of an interface for compatibility between the old and new optimizers. Let both the old and new optimizer planner inherit from this interface. One is NereidsPlanner and the other is OriginPlanner.
   Each implements its own plan function, and the return value is List<PlanFragment> .
   
   At present, although this adaptor mode can be implemented, the abstraction is not good.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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


[GitHub] [doris] morrySnow commented on a diff in pull request #10304: [feature](nereids) Integrate nereids into current SQL process framework

Posted by GitBox <gi...@apache.org>.
morrySnow commented on code in PR #10304:
URL: https://github.com/apache/doris/pull/10304#discussion_r902567680


##########
fe/fe-core/src/main/java/org/apache/doris/qe/SessionVariable.java:
##########
@@ -471,6 +473,15 @@ public class SessionVariable implements Serializable, Writable {
     @VariableMgr.VarAttr(name = ENABLE_ARRAY_TYPE)
     private boolean enableArrayType = false;
 
+    /**
+     * as the new optimizer is not mature yet, use this var
+     * to control whether to use new optimizer, remove it when
+     * the new optimizer is fully developed. I hope that day
+     * would be coming soon.
+     */
+    @VariableMgr.VarAttr(name = ENABLE_NEREIDS)
+    private boolean enableNereids = true;

Review Comment:
   enableNereids
   ```suggestion
       private boolean enableNereids = false;
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/parser/NereidsParser.java:
##########
@@ -0,0 +1,108 @@
+// 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.parser;
+
+import org.apache.doris.analysis.StatementBase;
+import org.apache.doris.nereids.DorisLexer;
+import org.apache.doris.nereids.DorisParser;
+import org.apache.doris.nereids.exceptions.ParsingException;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlanAdapter;
+
+import org.antlr.v4.runtime.CharStreams;
+import org.antlr.v4.runtime.CommonTokenStream;
+import org.antlr.v4.runtime.ParserRuleContext;
+import org.antlr.v4.runtime.atn.PredictionMode;
+import org.antlr.v4.runtime.misc.ParseCancellationException;
+
+import java.util.ArrayList;
+import java.util.List;
+import java.util.function.Function;
+
+/**
+ * Sql parser, convert sql DSL to logical plan.
+ */
+public class NereidsParser {
+
+    /**
+     * In MySQL protocol, client could send multi-statement in.
+     * a single packet.
+     * https://dev.mysql.com/doc/internals/en/com-set-option.html
+     */
+    public List<StatementBase> parseSQL(String originStr) throws Exception {
+        List<LogicalPlan> logicalPlanList = parseMultiple(originStr);
+        List<StatementBase> statementBaseList = new ArrayList<>();
+        for (LogicalPlan logicalPlan : logicalPlanList) {
+            LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalPlan);
+            statementBaseList.add(logicalPlanAdapter);
+        }
+        return statementBaseList;
+    }
+
+    /**
+     * parse sql DSL string.
+     *
+     * @param sql sql string
+     * @return logical plan
+     */
+    public LogicalPlan parseSingle(String sql) throws Exception {

Review Comment:
   moving in front of `parseMultiple` is better



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


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