Skip to content

Commit

Permalink
[feature](nereids) Integrate nereids into current SQL process framewo…
Browse files Browse the repository at this point in the history
…rk (apache#10304)

To integrate the nereids optimizer with new SQLParser and Planner to existing SQL process framework, I abstract a interface which named "Planner" and let the Both planner from nereids and stale optimizer implement it, to disguish it
with origin Planner, I rename the Planner to OriginalPlanner.

As we don't want to impact the existing logic too much, I defined a LogicalPlanAdapter to adapt the logicalPlan that is the output of the new paser to the existing code.

Besides, as the MySQL protocol supports sending multiple statements in one packet, so I add Nereids#SparseSQL method to handle this properly.
  • Loading branch information
Kikyou1997 authored Jun 24, 2022
1 parent 8f2b2b4 commit e82d843
Show file tree
Hide file tree
Showing 27 changed files with 817 additions and 599 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -35,10 +35,10 @@
import org.apache.doris.planner.DataPartition;
import org.apache.doris.planner.OlapScanNode;
import org.apache.doris.planner.OlapTableSink;
import org.apache.doris.planner.OriginalPlanner;
import org.apache.doris.planner.PlanFragment;
import org.apache.doris.planner.PlanFragmentId;
import org.apache.doris.planner.PlanNodeId;
import org.apache.doris.planner.Planner;
import org.apache.doris.planner.ScanNode;

import com.google.common.base.Preconditions;
Expand All @@ -49,7 +49,7 @@
import java.util.Map;


public class UpdatePlanner extends Planner {
public class UpdatePlanner extends OriginalPlanner {

private final IdGenerator<PlanNodeId> nodeIdGenerator = PlanNodeId.createGenerator();
private final IdGenerator<PlanFragmentId> fragmentIdGenerator =
Expand All @@ -65,6 +65,7 @@ public class UpdatePlanner extends Planner {

public UpdatePlanner(long dbId, OlapTable targetTable, List<Expr> setExprs,
TupleDescriptor srcTupleDesc, Analyzer analyzer) {
super(analyzer);
this.targetDBId = dbId;
this.targetTable = targetTable;
this.setExprs = setExprs;
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,27 +17,64 @@

package org.apache.doris.nereids;

import org.apache.doris.analysis.StatementBase;
import org.apache.doris.common.AnalysisException;
import org.apache.doris.common.UserException;
import org.apache.doris.common.util.VectorizedUtil;
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.PhysicalPlanTranslator;
import org.apache.doris.nereids.trees.plans.Plan;
import org.apache.doris.nereids.trees.plans.PlanContext;
import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
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 com.google.common.collect.Lists;

import java.util.ArrayList;
import java.util.List;

/**
* Planner to do query plan in Nereids.
*/
public class Planner {
public class NereidsPlanner extends Planner {

private PlannerContext plannerContext;
private final ConnectContext ctx;
private List<ScanNode> scanNodeList = null;

public NereidsPlanner(ConnectContext ctx) {
this.ctx = ctx;
}

@Override
public void plan(StatementBase queryStmt,
org.apache.doris.thrift.TQueryOptions queryOptions) throws UserException {
if (!(queryStmt instanceof LogicalPlanAdapter)) {
throw new RuntimeException("Wrong type of queryStmt, expected: <? extends LogicalPlanAdapter>");
}
LogicalPlanAdapter logicalPlanAdapter = (LogicalPlanAdapter) queryStmt;
PhysicalPlan physicalPlan = 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.setOutputExprs(queryStmt.getResultExprs());
if (VectorizedUtil.isVectorized()) {
root.getPlanRoot().convertToVectoriezd();
}
scanNodeList = planContext.getScanNodeList();
}

/**
* Do analyze and optimize for query plan.
Expand Down Expand Up @@ -67,6 +104,11 @@ public PhysicalPlan plan(LogicalPlan plan, PhysicalProperties outputProperties,
return getRoot().extractPlan();
}

@Override
public List<ScanNode> getScanNodes() {
return scanNodeList;
}

public Group getRoot() {
return plannerContext.getOptimizerContext().getMemo().getRoot();
}
Expand All @@ -93,4 +135,9 @@ private PhysicalPlan chooseBestPlan(Group rootGroup, PhysicalProperties physical

return physicalPlan;
}

@Override
public boolean isBlockQuery() {
return true;
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -37,6 +37,7 @@
import org.apache.doris.nereids.DorisParser.JoinRelationContext;
import org.apache.doris.nereids.DorisParser.LogicalBinaryContext;
import org.apache.doris.nereids.DorisParser.LogicalNotContext;
import org.apache.doris.nereids.DorisParser.MultiStatementsContext;
import org.apache.doris.nereids.DorisParser.MultipartIdentifierContext;
import org.apache.doris.nereids.DorisParser.NamedExpressionContext;
import org.apache.doris.nereids.DorisParser.NamedExpressionSeqContext;
Expand All @@ -52,6 +53,7 @@
import org.apache.doris.nereids.DorisParser.SingleStatementContext;
import org.apache.doris.nereids.DorisParser.SortItemContext;
import org.apache.doris.nereids.DorisParser.StarContext;
import org.apache.doris.nereids.DorisParser.StatementContext;
import org.apache.doris.nereids.DorisParser.StringLiteralContext;
import org.apache.doris.nereids.DorisParser.TableNameContext;
import org.apache.doris.nereids.DorisParser.WhereClauseContext;
Expand Down Expand Up @@ -147,6 +149,18 @@ public LogicalPlan visitSingleStatement(SingleStatementContext ctx) {
return ParserUtils.withOrigin(ctx, f);
}

/**
* Visit multi-statements.
*/
public Object visitMultiStatements(MultiStatementsContext ctx) {
List<LogicalPlan> logicalPlanList = new ArrayList<>();
for (StatementContext stmtCtx : ctx.statement()) {
LogicalPlan logicalPlan = (LogicalPlan) visit(stmtCtx);
logicalPlanList.add(logicalPlan);
}
return logicalPlanList;
}

/* ********************************************************************************************
* Plan parsing
* ******************************************************************************************** */
Expand Down
Original file line number Diff line number Diff line change
@@ -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 {
return (LogicalPlan) parse(sql, DorisParser::singleStatement);
}

public List<LogicalPlan> parseMultiple(String sql) throws Exception {
return (List<LogicalPlan>) parse(sql, DorisParser::multiStatements);
}

private Object parse(String sql, Function<DorisParser, ParserRuleContext> parseFunction) {
try {
ParserRuleContext tree = toAst(sql, parseFunction);
LogicalPlanBuilder logicalPlanBuilder = new LogicalPlanBuilder();
return logicalPlanBuilder.visit(tree);
} catch (StackOverflowError e) {
throw new ParsingException(e.getMessage());
}
}

private ParserRuleContext toAst(String sql, Function<DorisParser, ParserRuleContext> parseFunction) {
DorisLexer lexer = new DorisLexer(new CaseInsensitiveStream(CharStreams.fromString(sql)));
CommonTokenStream tokenStream = new CommonTokenStream(lexer);
DorisParser parser = new DorisParser(tokenStream);
// parser.addParseListener(PostProcessor)
// parser.removeErrorListeners()
// parser.addErrorListener(ParseErrorListener)
ParserRuleContext tree;
try {
// first, try parsing with potentially faster SLL mode
parser.getInterpreter().setPredictionMode(PredictionMode.SLL);
tree = parseFunction.apply(parser);
} catch (ParseCancellationException ex) {
// if we fail, parse with LL mode
tokenStream.seek(0); // rewind input stream
parser.reset();

parser.getInterpreter().setPredictionMode(PredictionMode.LL);
tree = parseFunction.apply(parser);
}
return tree;
}

public Expression createExpression(String expression) {
return (Expression) parse(expression, DorisParser::expression);
}
}

This file was deleted.

Original file line number Diff line number Diff line change
Expand Up @@ -23,6 +23,8 @@
public class PhysicalProperties {
private DistributionSpec distributionDesc;

public PhysicalProperties() {}

public DistributionSpec getDistributionDesc() {
return distributionDesc;
}
Expand Down
Loading

0 comments on commit e82d843

Please sign in to comment.