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

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


The following commit(s) were added to refs/heads/master by this push:
     new 972941b0c15 [fix](prepare statement)Prepare stage only explain sql to 
get output slots. (#56224)
972941b0c15 is described below

commit 972941b0c150b9de3ba561b292e1bb0836a61ae1
Author: James <[email protected]>
AuthorDate: Sat Sep 20 15:16:46 2025 +0800

    [fix](prepare statement)Prepare stage only explain sql to get output slots. 
(#56224)
    
    Prepare stage only explain sql to get output slots. After explain, we
    can get all the output slots info, meanwhile, we don't need to go
    through all the other plan stages. This could solve some errors in
    during plan.
---
 .../org/apache/doris/nereids/NereidsPlanner.java   |   2 +-
 .../nereids/rules/analysis/ExpressionAnalyzer.java |   4 +-
 .../LogicalResultSinkToShortCircuitPointQuery.java |   4 +-
 .../nereids/trees/expressions/Placeholder.java     |   4 +-
 .../functions/scalar/PreparePlaceholder.java       |  73 ---------------------
 .../expressions/visitor/ScalarFunctionVisitor.java |   5 --
 .../nereids/trees/plans/PrepareCommandPlanner.java |  39 +++++++++++
 .../java/org/apache/doris/qe/StmtExecutor.java     |   5 +-
 .../data/prepared_stmt_p0/prepared_stmt.out        | Bin 5389 -> 5414 bytes
 .../suites/prepared_stmt_p0/prepared_stmt.groovy   |  29 ++++++++
 10 files changed, 76 insertions(+), 89 deletions(-)

diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
index 33aa3ddce4c..846aa9019af 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/NereidsPlanner.java
@@ -983,7 +983,7 @@ public class NereidsPlanner extends Planner {
         return PhysicalProperties.GATHER;
     }
 
-    private ExplainLevel getExplainLevel(ExplainOptions explainOptions) {
+    protected ExplainLevel getExplainLevel(ExplainOptions explainOptions) {
         if (explainOptions == null) {
             return ExplainLevel.NONE;
         }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
index d6f6034cc6c..040224f7c54 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/ExpressionAnalyzer.java
@@ -74,13 +74,11 @@ import 
org.apache.doris.nereids.trees.expressions.functions.agg.AggregateFunctio
 import 
org.apache.doris.nereids.trees.expressions.functions.agg.NullableAggregateFunction;
 import 
org.apache.doris.nereids.trees.expressions.functions.agg.SupportMultiDistinct;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Lambda;
-import 
org.apache.doris.nereids.trees.expressions.functions.scalar.PreparePlaceholder;
 import 
org.apache.doris.nereids.trees.expressions.functions.udf.AliasUdfBuilder;
 import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdaf;
 import org.apache.doris.nereids.trees.expressions.functions.udf.JavaUdf;
 import org.apache.doris.nereids.trees.expressions.functions.udf.UdfBuilder;
 import org.apache.doris.nereids.trees.expressions.literal.IntegerLikeLiteral;
-import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.Literal;
 import org.apache.doris.nereids.trees.expressions.literal.NullLiteral;
 import org.apache.doris.nereids.trees.expressions.literal.StringLiteral;
@@ -652,7 +650,7 @@ public class ExpressionAnalyzer extends 
SubExprAnalyzer<ExpressionRewriteContext
         // In prepare stage, the realExpr has not been set, set it to 
StringLiteral so that we can plan the statement
         // and get the output slots in prepare stage, which is required by 
Mysql api definition.
         if (realExpr == null && 
context.cascadesContext.getStatementContext().isPrepareStage()) {
-            realExpr = new PreparePlaceholder(new IntegerLiteral(1));
+            realExpr = new 
StringLiteral(String.valueOf(placeholder.getPlaceholderId().asInt()));
         }
         return visit(realExpr, context);
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LogicalResultSinkToShortCircuitPointQuery.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LogicalResultSinkToShortCircuitPointQuery.java
index 99e74357a17..dfcd2ea289c 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LogicalResultSinkToShortCircuitPointQuery.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/rewrite/LogicalResultSinkToShortCircuitPointQuery.java
@@ -26,7 +26,6 @@ import org.apache.doris.nereids.trees.expressions.Cast;
 import org.apache.doris.nereids.trees.expressions.EqualTo;
 import org.apache.doris.nereids.trees.expressions.Expression;
 import org.apache.doris.nereids.trees.expressions.SlotReference;
-import 
org.apache.doris.nereids.trees.expressions.functions.scalar.PreparePlaceholder;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.logical.LogicalFilter;
 import org.apache.doris.nereids.trees.plans.logical.LogicalOlapScan;
@@ -58,8 +57,7 @@ public class LogicalResultSinkToShortCircuitPointQuery 
implements RewriteRuleFac
                         && 
(removeCast(expression.child(0)).isKeyColumnFromTable()
                         || (expression.child(0) instanceof SlotReference
                         && ((SlotReference) 
expression.child(0)).getName().equals(Column.DELETE_SIGN)))
-                        && (expression.child(1).isLiteral()
-                        || removeCast(expression.child(1)) instanceof 
PreparePlaceholder));
+                        && expression.child(1).isLiteral());
     }
 
     private boolean scanMatchShortCircuitCondition(LogicalOlapScan olapScan) {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Placeholder.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Placeholder.java
index 27769750ad0..caa3fd07e30 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Placeholder.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/Placeholder.java
@@ -24,7 +24,7 @@ import 
org.apache.doris.nereids.trees.expressions.shape.LeafExpression;
 import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
 import org.apache.doris.nereids.trees.plans.PlaceholderId;
 import org.apache.doris.nereids.types.DataType;
-import org.apache.doris.nereids.types.NullType;
+import org.apache.doris.nereids.types.StringType;
 
 import java.util.Optional;
 
@@ -78,7 +78,7 @@ public class Placeholder extends Expression implements 
LeafExpression {
 
     @Override
     public DataType getDataType() throws UnboundException {
-        return NullType.INSTANCE;
+        return StringType.INSTANCE;
     }
 
     public Placeholder withNewMysqlColType(int mysqlTypeCode) {
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/PreparePlaceholder.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/PreparePlaceholder.java
deleted file mode 100644
index cacd19e4872..00000000000
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/functions/scalar/PreparePlaceholder.java
+++ /dev/null
@@ -1,73 +0,0 @@
-// Licensed to the Apache Software Foundation (ASF) under one
-// or more contributor license agreements.  See the NOTICE file
-// distributed with this work for additional information
-// regarding copyright ownership.  The ASF licenses this file
-// to you under the Apache License, Version 2.0 (the
-// "License"); you may not use this file except in compliance
-// with the License.  You may obtain a copy of the License at
-//
-//   http://www.apache.org/licenses/LICENSE-2.0
-//
-// Unless required by applicable law or agreed to in writing,
-// software distributed under the License is distributed on an
-// "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
-// KIND, either express or implied.  See the License for the
-// specific language governing permissions and limitations
-// under the License.
-
-package org.apache.doris.nereids.trees.expressions.functions.scalar;
-
-import org.apache.doris.catalog.FunctionSignature;
-import org.apache.doris.nereids.trees.expressions.Expression;
-import 
org.apache.doris.nereids.trees.expressions.functions.ExplicitlyCastableSignature;
-import org.apache.doris.nereids.trees.expressions.functions.PropagateNullable;
-import org.apache.doris.nereids.trees.expressions.visitor.ExpressionVisitor;
-import org.apache.doris.nereids.types.IntegerType;
-import org.apache.doris.nereids.types.StringType;
-
-import com.google.common.collect.ImmutableList;
-
-import java.util.List;
-
-/**
- * Internal function for prepare statement only.
- */
-public class PreparePlaceholder extends ScalarFunction
-        implements ExplicitlyCastableSignature, PropagateNullable {
-
-    public static final List<FunctionSignature> SIGNATURES = ImmutableList.of(
-            
FunctionSignature.ret(StringType.INSTANCE).args(IntegerType.INSTANCE)
-    );
-
-    /**
-     * constructor with 1 argument.
-     */
-    public PreparePlaceholder(Expression arg0) {
-        super("prepare_placeholder", arg0);
-    }
-
-    @Override
-    public boolean foldable() {
-        return false;
-    }
-
-    @Override
-    public List<FunctionSignature> getSignatures() {
-        return SIGNATURES;
-    }
-
-    @Override
-    public <R, C> R accept(ExpressionVisitor<R, C> visitor, C context) {
-        return visitor.visitPreparePlaceholer(this, context);
-    }
-
-    @Override
-    public boolean equals(Object o) {
-        return this == o;
-    }
-
-    @Override
-    public int hashCode() {
-        return System.identityHashCode(this);
-    }
-}
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
index f844b077c52..f3077362cb8 100644
--- 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/expressions/visitor/ScalarFunctionVisitor.java
@@ -369,7 +369,6 @@ import 
org.apache.doris.nereids.trees.expressions.functions.scalar.Pmod;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Positive;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Pow;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Power;
-import 
org.apache.doris.nereids.trees.expressions.functions.scalar.PreparePlaceholder;
 import org.apache.doris.nereids.trees.expressions.functions.scalar.Protocol;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantilePercent;
 import 
org.apache.doris.nereids.trees.expressions.functions.scalar.QuantileStateEmpty;
@@ -1823,10 +1822,6 @@ public interface ScalarFunctionVisitor<R, C> {
         return visitScalarFunction(parseDataSize, context);
     }
 
-    default R visitPreparePlaceholer(PreparePlaceholder preparePlaceholder, C 
context) {
-        return visitScalarFunction(preparePlaceholder, context);
-    }
-
     default R visitPassword(Password password, C context) {
         return visitScalarFunction(password, context);
     }
diff --git 
a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PrepareCommandPlanner.java
 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PrepareCommandPlanner.java
new file mode 100644
index 00000000000..65ad7cd1bb9
--- /dev/null
+++ 
b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PrepareCommandPlanner.java
@@ -0,0 +1,39 @@
+// 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;
+
+import org.apache.doris.analysis.ExplainOptions;
+import org.apache.doris.nereids.NereidsPlanner;
+import org.apache.doris.nereids.StatementContext;
+import 
org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel;
+
+/**
+ * Planner for prepare command. Used for PrepareCommand to get output slots 
metadata.
+ * In prepare stage, we only need to do analyze stage of plan to get the 
output slots metadata.
+ * And we skip all the other stages to avoid errors.
+ */
+public class PrepareCommandPlanner extends NereidsPlanner {
+    public PrepareCommandPlanner(StatementContext statementContext) {
+        super(statementContext);
+    }
+
+    protected ExplainLevel getExplainLevel(ExplainOptions explainOptions) {
+        return ExplainLevel.ANALYZED_PLAN;
+    }
+
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java 
b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
index a10bf79fca8..c3958bee85d 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/qe/StmtExecutor.java
@@ -84,6 +84,7 @@ import org.apache.doris.nereids.trees.expressions.Slot;
 import org.apache.doris.nereids.trees.expressions.SlotReference;
 import org.apache.doris.nereids.trees.expressions.literal.DateTimeV2Literal;
 import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.PrepareCommandPlanner;
 import org.apache.doris.nereids.trees.plans.algebra.InlineTable;
 import org.apache.doris.nereids.trees.plans.commands.Command;
 import org.apache.doris.nereids.trees.plans.commands.CreateTableCommand;
@@ -1854,9 +1855,9 @@ public class StmtExecutor {
         Preconditions.checkState(parsedStmt instanceof LogicalPlanAdapter,
                 "Nereids only process LogicalPlanAdapter,"
                         + " but parsedStmt is " + 
parsedStmt.getClass().getName());
-        NereidsPlanner nereidsPlanner = new NereidsPlanner(statementContext);
+        NereidsPlanner nereidsPlanner = new 
PrepareCommandPlanner(statementContext);
         nereidsPlanner.plan(parsedStmt, 
context.getSessionVariable().toThrift());
-        return nereidsPlanner.getPhysicalPlan().getOutput();
+        return 
nereidsPlanner.getCascadesContext().getRewritePlan().getOutput();
     }
 
     public List<ResultRow> executeInternalQuery() {
diff --git a/regression-test/data/prepared_stmt_p0/prepared_stmt.out 
b/regression-test/data/prepared_stmt_p0/prepared_stmt.out
index a940c924e64..f5d8a45d6fc 100644
Binary files a/regression-test/data/prepared_stmt_p0/prepared_stmt.out and 
b/regression-test/data/prepared_stmt_p0/prepared_stmt.out differ
diff --git a/regression-test/suites/prepared_stmt_p0/prepared_stmt.groovy 
b/regression-test/suites/prepared_stmt_p0/prepared_stmt.groovy
index 98dce5ec8c6..8c0cbd5cdff 100644
--- a/regression-test/suites/prepared_stmt_p0/prepared_stmt.groovy
+++ b/regression-test/suites/prepared_stmt_p0/prepared_stmt.groovy
@@ -327,6 +327,35 @@ suite("test_prepared_stmt", "nonConcurrent") {
         stmt_read.setString(1, "2025-08-15 11:22:33")
         stmt_read.setString(2, "DAY")
         qe_select25 stmt_read
+
+        sql """drop table if exists 
table_20_undef_partitions2_keys3_properties4_distributed_by54"""
+        sql """ CREATE TABLE IF NOT EXISTS 
`table_20_undef_partitions2_keys3_properties4_distributed_by54` (
+              `col_int_undef_signed` int NULL,
+              `col_int_undef_signed2` int NULL,
+              `pk` int NULL
+            ) ENGINE=OLAP
+            DUPLICATE KEY(`col_int_undef_signed`, `col_int_undef_signed2`, 
`pk`)
+            DISTRIBUTED BY HASH(`pk`) BUCKETS 10
+            PROPERTIES (
+            "replication_allocation" = "tag.location.default: 1",
+            "min_load_replica_num" = "-1",
+            "is_being_synced" = "false",
+            "storage_medium" = "hdd",
+            "storage_format" = "V2",
+            "inverted_index_storage_format" = "V2",
+            "light_schema_change" = "true",
+            "disable_auto_compaction" = "false",
+            "enable_single_replica_compaction" = "false",
+            "group_commit_interval_ms" = "10000",
+            "group_commit_data_bytes" = "134217728"
+            ); 
+        """
+        sql """insert into 
table_20_undef_partitions2_keys3_properties4_distributed_by54 values (1, 1, 1), 
(2, 2, 2)"""
+        stmt_read = prepareStatement "select min ( pk - ? ) pk , pk as pk from 
table_20_undef_partitions2_keys3_properties4_distributed_by54 tbl_alias1 group 
by pk having ( pk >= pk ) or ( round ( sign ( sign ( pk ) ) ) - ? < ? ) order 
by pk "
+        stmt_read.setString(1, "1")
+        stmt_read.setString(2, "2")
+        stmt_read.setString(3, "3")
+        qe_select26 stmt_read
     }
 
     // test stmtId overflow


---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]

Reply via email to