morrySnow commented on code in PR #35318:
URL: https://github.com/apache/doris/pull/35318#discussion_r1620306454


##########
fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java:
##########
@@ -116,6 +117,11 @@ public class StatementContext implements Closeable {
     private final Set<String> viewDdlSqlSet = Sets.newHashSet();
     private final SqlCacheContext sqlCacheContext;
 
+    // generate for next id for prepared statement's placeholders, which is 
connection level
+    private final IdGenerator<RelationId> placeHolderRelationIdGenerator = 
RelationId.createGenerator();

Review Comment:
   do not use RelationId, u should create a new id type named `PlaceHolderId`



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Placeholder.java:
##########
@@ -0,0 +1,74 @@
+// 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.expressions;
+
+import org.apache.doris.catalog.MysqlColType;
+import org.apache.doris.nereids.exceptions.UnboundException;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.trees.plans.RelationId;
+import org.apache.doris.nereids.types.DataType;
+
+/**
+ * Placeholder for prepared statement
+ */
+public class Placeholder extends Expression {
+    private final RelationId exprId;

Review Comment:
   ```suggestion
       private final PlaceholderId placeholderId;
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Placeholder.java:
##########
@@ -0,0 +1,74 @@
+// 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.expressions;
+
+import org.apache.doris.catalog.MysqlColType;
+import org.apache.doris.nereids.exceptions.UnboundException;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.trees.plans.RelationId;
+import org.apache.doris.nereids.types.DataType;
+
+/**
+ * Placeholder for prepared statement
+ */
+public class Placeholder extends Expression {
+    private final RelationId exprId;
+    private final MysqlColType mysqlColType;
+
+    public Placeholder(RelationId exprId) {
+        this.exprId = exprId;
+        this.mysqlColType = null;

Review Comment:
   use Optional if it could be null



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ExpressionVisitor.java:
##########
@@ -505,6 +506,10 @@ public R visitMatchPhraseEdge(MatchPhraseEdge 
matchPhraseEdge, C context) {
         return visitMatch(matchPhraseEdge, context);
     }
 
+    public R visitPlaceholderExpr(Placeholder placeholderExpr, C context) {

Review Comment:
   ```suggestion
       public R visitPlaceholder(Placeholder placeholder, C context) {
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Placeholder.java:
##########
@@ -0,0 +1,74 @@
+// 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.expressions;
+
+import org.apache.doris.catalog.MysqlColType;
+import org.apache.doris.nereids.exceptions.UnboundException;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.trees.plans.RelationId;
+import org.apache.doris.nereids.types.DataType;
+
+/**
+ * Placeholder for prepared statement
+ */
+public class Placeholder extends Expression {
+    private final RelationId exprId;
+    private final MysqlColType mysqlColType;
+
+    public Placeholder(RelationId exprId) {
+        this.exprId = exprId;
+        this.mysqlColType = null;
+    }
+
+    public Placeholder(RelationId exprId, MysqlColType mysqlColType) {
+        this.exprId = exprId;
+        this.mysqlColType = mysqlColType;
+    }
+
+    public RelationId getExprId() {
+        return exprId;
+    }
+
+    @Override
+    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
+        return visitor.visitPlaceholderExpr(this, context);
+    }
+
+    @Override
+    public boolean nullable() {
+        return true;
+    }
+
+    @Override
+    public String toSql() {
+        return "?";
+    }
+
+    @Override
+    public DataType getDataType() throws UnboundException {
+        return null;

Review Comment:
   do not return null. return NullType.Instance of throw Exception



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();
+    private final LogicalPlan inner;
+
+    private final String stmtName;
+
+    private final OriginStatement originalStmt;
+
+    /**
+     * constructor
+     * @param name the statement name which represents statement id for 
prepared statement
+     * @param plan the inner statement
+     * @param placeholders the parameters for this prepared statement
+     * @param originalStmt original statement from StmtExecutor
+     */
+    public PreparedCommand(String name, LogicalPlan plan, List<Placeholder> 
placeholders,
+                OriginStatement originalStmt) {
+        super(PlanType.UNKNOWN);
+        this.inner = plan;
+        if (placeholders != null) {
+            this.params = placeholders;
+        }

Review Comment:
   let placeholders always not null and then
   ```suggestion
           this.params.addAll(placeholders);
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();

Review Comment:
   ```suggestion
       protected List<Placeholder> placeholders = new ArrayList<>();
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();
+    private final LogicalPlan inner;
+
+    private final String stmtName;

Review Comment:
   ```suggestion
       private final String name;
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExecuteCommand.java:
##########
@@ -0,0 +1,83 @@
+// 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.commands;
+
+import org.apache.doris.nereids.StatementContext;
+import org.apache.doris.nereids.exceptions.AnalysisException;
+import org.apache.doris.nereids.glue.LogicalPlanAdapter;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import java.util.List;
+import java.util.stream.Collectors;
+
+/**
+ * Prepared Statement
+ */
+public class ExecuteCommand extends Command {
+    private final String stmtName;
+    private final PreparedCommand preparedCommand;
+    private final StatementContext statementContext;
+
+    public ExecuteCommand(String stmtName, PreparedCommand preparedCommand, 
StatementContext statementContext) {
+        super(PlanType.UNKNOWN);

Review Comment:
   ```suggestion
           super(PlanType.EXECUTE_COMMAND);
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();
+    private final LogicalPlan inner;
+
+    private final String stmtName;
+
+    private final OriginStatement originalStmt;
+
+    /**
+     * constructor
+     * @param name the statement name which represents statement id for 
prepared statement
+     * @param plan the inner statement
+     * @param placeholders the parameters for this prepared statement
+     * @param originalStmt original statement from StmtExecutor
+     */
+    public PreparedCommand(String name, LogicalPlan plan, List<Placeholder> 
placeholders,
+                OriginStatement originalStmt) {
+        super(PlanType.UNKNOWN);
+        this.inner = plan;
+        if (placeholders != null) {
+            this.params = placeholders;
+        }
+        this.stmtName = name;
+        this.originalStmt = originalStmt;
+    }
+
+    public String getName() {
+        return stmtName;
+    }
+
+    public List<Placeholder> params() {
+        return params;
+    }
+
+    public int getParamLen() {
+        if (params == null) {
+            return 0;
+        }
+        return params.size();
+    }
+
+    public LogicalPlan getInnerPlan() {
+        return inner;
+    }
+
+    public OriginStatement getOriginalStmt() {
+        return originalStmt;
+    }
+
+    /**
+     * return the labels of paramters
+     */
+    public List<String> getLabels() {
+        List<String> labels = new ArrayList<>();
+        if (params == null) {
+            return labels;
+        }
+        for (Placeholder parameter : params) {
+            labels.add("$" + parameter.getExprId().asInt());
+        }
+        return labels;
+    }
+
+    // register prepared statement with attached statement id
+    @Override
+    public void run(ConnectContext ctx, StmtExecutor executor) throws 
Exception {
+        List<String> labels = getLabels();
+        // register prepareStmt
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("add prepared statement {}, isBinaryProtocol {}",
+                    stmtName, ctx.getCommand() == 
MysqlCommand.COM_STMT_PREPARE);
+        }
+        ctx.addPreparedStatementContext(stmtName,
+                new PreparedStatementContext(this, ctx, 
ctx.getStatementContext(), stmtName));
+        if (ctx.getCommand() == MysqlCommand.COM_STMT_PREPARE) {
+            executor.sendStmtPrepareOK((int) ctx.getStmtId(), labels);
+        }
+    }
+
+    @Override
+    public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+        return visitor.visit(this, context);
+    }
+
+    public PreparedCommand withNewPreparedCommand(List<Placeholder> params) {

Review Comment:
   ```suggestion
       public PreparedCommand withPlaceholders(List<Placeholder> params) {
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java:
##########
@@ -541,6 +542,13 @@ public Expression visitNot(Not not, 
ExpressionRewriteContext context) {
         return expr;
     }
 
+    @Override
+    public Expression visitPlaceholderExpr(Placeholder placeholderExpr, 
ExpressionRewriteContext context) {

Review Comment:
   ```suggestion
       public Expression visitPlaceholder(Placeholder placeholder, 
ExpressionRewriteContext context) {
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/StatementContext.java:
##########
@@ -465,6 +482,14 @@ public void close() {
         releasePlannerResources();
     }
 
+    public List<Placeholder> getParams() {

Review Comment:
   param is not a good name. it means diff things in diff context. rename it to 
`getPlaceHolders`



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java:
##########
@@ -1003,6 +1008,11 @@ public List<Pair<LogicalPlan, StatementContext>> 
visitMultiStatements(MultiState
             }
             logicalPlans.add(Pair.of(
                     ParserUtils.withOrigin(ctx, () -> (LogicalPlan) 
visit(statement)), statementContext));
+            if (tokenPosToParameters != null) {

Review Comment:
   could we let it always not null, then we no need to process null in all place



##########
regression-test/suites/point_query_p0/test_point_query.groovy:
##########
@@ -30,7 +30,7 @@ suite("test_point_query") {
     try {
         set_be_config.call("disable_storage_row_cache", "false")
         // nereids do not support point query now
-        sql """set enable_nereids_planner=false"""
+        sql """set global enable_nereids_planner=false"""

Review Comment:
   why need set global?



##########
fe/fe-core/src/main/java/org/apache/doris/qe/ConnectProcessor.java:
##########
@@ -257,8 +257,9 @@ public void executeQuery(MysqlCommand mysqlCommand, String 
originStmt) throws Ex
         Exception nereidsSyntaxException = null;
         long parseSqlStartTime = System.currentTimeMillis();
         List<StatementBase> cachedStmts = null;
-        // Nereids do not support prepare and execute now, so forbid prepare 
command, only process query command
-        if (mysqlCommand == MysqlCommand.COM_QUERY && 
sessionVariable.isEnableNereidsPlanner()) {
+        boolean nereidsUseServerPrep = 
sessionVariable.enableServeSidePreparedStatement
+                        || mysqlCommand == MysqlCommand.COM_QUERY;

Review Comment:
   add a todo comemnt, remove nereidsUseServerPrep after Nereids support full 
prepare



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Placeholder.java:
##########
@@ -0,0 +1,74 @@
+// 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.expressions;
+
+import org.apache.doris.catalog.MysqlColType;
+import org.apache.doris.nereids.exceptions.UnboundException;
+import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
+import org.apache.doris.nereids.trees.plans.RelationId;
+import org.apache.doris.nereids.types.DataType;
+
+/**
+ * Placeholder for prepared statement
+ */
+public class Placeholder extends Expression {

Review Comment:
   ```suggestion
   public class Placeholder extends Expression implements LeafExpression {
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/qe/MysqlConnectProcessor.java:
##########
@@ -159,8 +146,101 @@ private void handleExecute() {
             ctx.getState().setError(ErrorCode.ERR_UNKNOWN_ERROR,
                     e.getClass().getSimpleName() + ", msg: " + e.getMessage());
         }
-        if (!stmtStr.isEmpty()) {
-            auditAfterExec(stmtStr, prepareCtx.stmt.getInnerStmt(), null, 
false);
+        auditAfterExec(stmtStr, executor.getParsedStmt(), 
executor.getQueryStatisticsForAuditLog(), true);
+    }
+
+    private void handleExecute(Command preparedCommand, long stmtId, 
PreparedStatementContext prepCtx) {

Review Comment:
   ```suggestion
       private void handleExecute(PrepareCommand preparedCommand, long stmtId, 
PreparedStatementContext prepCtx) {
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();

Review Comment:
   why protected? why not final?



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();
+    private final LogicalPlan inner;
+
+    private final String stmtName;
+
+    private final OriginStatement originalStmt;
+
+    /**
+     * constructor
+     * @param name the statement name which represents statement id for 
prepared statement
+     * @param plan the inner statement
+     * @param placeholders the parameters for this prepared statement
+     * @param originalStmt original statement from StmtExecutor
+     */
+    public PreparedCommand(String name, LogicalPlan plan, List<Placeholder> 
placeholders,
+                OriginStatement originalStmt) {
+        super(PlanType.UNKNOWN);

Review Comment:
   ```suggestion
           super(PlanType.PREPARE_COMMAND);
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();
+    private final LogicalPlan inner;

Review Comment:
   ```suggestion
       private final LogicalPlan logicalPlan;
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();
+    private final LogicalPlan inner;
+
+    private final String stmtName;
+
+    private final OriginStatement originalStmt;
+
+    /**
+     * constructor
+     * @param name the statement name which represents statement id for 
prepared statement
+     * @param plan the inner statement
+     * @param placeholders the parameters for this prepared statement
+     * @param originalStmt original statement from StmtExecutor
+     */
+    public PreparedCommand(String name, LogicalPlan plan, List<Placeholder> 
placeholders,

Review Comment:
   ```suggestion
       public PrepareCommand(String name, LogicalPlan plan, List<Placeholder> 
placeholders,
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/qe/PreparedStatementContext.java:
##########
@@ -0,0 +1,51 @@
+// 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.qe;
+
+import org.apache.doris.nereids.StatementContext;
+import org.apache.doris.nereids.trees.plans.commands.Command;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class PreparedStatementContext {
+    private static final Logger LOG = 
LogManager.getLogger(PrepareStmtContext.class);
+    public Command command;

Review Comment:
   ```suggestion
       public PrepareCommand command;
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/PreparedCommand.java:
##########
@@ -0,0 +1,128 @@
+// 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.commands;
+
+import org.apache.doris.mysql.MysqlCommand;
+import org.apache.doris.nereids.trees.expressions.Placeholder;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.OriginStatement;
+import org.apache.doris.qe.PreparedStatementContext;
+import org.apache.doris.qe.StmtExecutor;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+import java.util.ArrayList;
+import java.util.List;
+
+/**
+ * Prepared Statement
+ */
+public class PreparedCommand extends Command {
+    private static final Logger LOG = LogManager.getLogger(StmtExecutor.class);
+
+    protected List<Placeholder> params = new ArrayList<>();
+    private final LogicalPlan inner;
+
+    private final String stmtName;
+
+    private final OriginStatement originalStmt;
+
+    /**
+     * constructor
+     * @param name the statement name which represents statement id for 
prepared statement
+     * @param plan the inner statement
+     * @param placeholders the parameters for this prepared statement
+     * @param originalStmt original statement from StmtExecutor
+     */
+    public PreparedCommand(String name, LogicalPlan plan, List<Placeholder> 
placeholders,
+                OriginStatement originalStmt) {
+        super(PlanType.UNKNOWN);
+        this.inner = plan;
+        if (placeholders != null) {
+            this.params = placeholders;
+        }
+        this.stmtName = name;
+        this.originalStmt = originalStmt;
+    }
+
+    public String getName() {
+        return stmtName;
+    }
+
+    public List<Placeholder> params() {
+        return params;
+    }
+
+    public int getParamLen() {
+        if (params == null) {
+            return 0;
+        }
+        return params.size();
+    }
+
+    public LogicalPlan getInnerPlan() {
+        return inner;
+    }
+
+    public OriginStatement getOriginalStmt() {
+        return originalStmt;
+    }
+
+    /**
+     * return the labels of paramters
+     */
+    public List<String> getLabels() {
+        List<String> labels = new ArrayList<>();
+        if (params == null) {
+            return labels;
+        }
+        for (Placeholder parameter : params) {
+            labels.add("$" + parameter.getExprId().asInt());
+        }
+        return labels;
+    }
+
+    // register prepared statement with attached statement id
+    @Override
+    public void run(ConnectContext ctx, StmtExecutor executor) throws 
Exception {
+        List<String> labels = getLabels();
+        // register prepareStmt
+        if (LOG.isDebugEnabled()) {
+            LOG.debug("add prepared statement {}, isBinaryProtocol {}",
+                    stmtName, ctx.getCommand() == 
MysqlCommand.COM_STMT_PREPARE);
+        }
+        ctx.addPreparedStatementContext(stmtName,
+                new PreparedStatementContext(this, ctx, 
ctx.getStatementContext(), stmtName));
+        if (ctx.getCommand() == MysqlCommand.COM_STMT_PREPARE) {
+            executor.sendStmtPrepareOK((int) ctx.getStmtId(), labels);
+        }

Review Comment:
   ```suggestion
           executor.sendStmtPrepareOK((int) ctx.getStmtId(), labels);
   ```



##########
fe/fe-core/src/main/java/org/apache/doris/qe/PreparedStatementContext.java:
##########
@@ -0,0 +1,51 @@
+// 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.qe;
+
+import org.apache.doris.nereids.StatementContext;
+import org.apache.doris.nereids.trees.plans.commands.Command;
+
+import org.apache.logging.log4j.LogManager;
+import org.apache.logging.log4j.Logger;
+
+public class PreparedStatementContext {
+    private static final Logger LOG = 
LogManager.getLogger(PrepareStmtContext.class);

Review Comment:
   remove useless log object



-- 
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-unsubscr...@doris.apache.org

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


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


Reply via email to