You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by kx...@apache.org on 2023/06/06 15:15:12 UTC

[doris] 10/36: [Feature](Nereids) support update unique table statement (#20313)

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

kxiao pushed a commit to branch branch-2.0-beta
in repository https://gitbox.apache.org/repos/asf/doris.git

commit d8c6dfbf09faf350b6f2139688c55fad3f6ff04c
Author: mch_ucchi <41...@users.noreply.github.com>
AuthorDate: Tue Jun 6 20:32:43 2023 +0800

    [Feature](Nereids) support update unique table statement (#20313)
---
 .../antlr4/org/apache/doris/nereids/DorisLexer.g4  |   1 +
 .../antlr4/org/apache/doris/nereids/DorisParser.g4 |  13 ++
 .../doris/nereids/parser/LogicalPlanBuilder.java   |  74 +++++++---
 .../doris/nereids/trees/plans/Explainable.java     |  27 ++++
 .../trees/plans/commands/ExplainCommand.java       |   9 +-
 .../plans/commands/InsertIntoTableCommand.java     |  19 ++-
 .../trees/plans/commands/UpdateCommand.java        | 156 +++++++++++++++++++++
 .../trees/plans/logical/AbstractLogicalPlan.java   |   9 +-
 .../trees/plans/physical/AbstractPhysicalPlan.java |   9 +-
 .../trees/plans/visitor/CommandVisitor.java        |  46 ++++++
 .../nereids/trees/plans/visitor/PlanVisitor.java   |  18 +--
 .../nereids/trees/plans/UpdateCommandTest.java     | 126 +++++++++++++++++
 .../data/nereids_p0/update/update_unique_table.out |   6 +-
 .../suites/nereids_p0/update/load.groovy           |  81 +++++++++++
 .../nereids_p0/update/update_unique_table.groovy   |  39 ++++++
 15 files changed, 586 insertions(+), 47 deletions(-)

diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
index 11a1c88618..3c54093cdd 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisLexer.g4
@@ -159,6 +159,7 @@ DATE_ADD: 'DATE_ADD';
 DATE_SUB: 'DATE_SUB';
 DATE_DIFF: 'DATE_DIFF';
 DBPROPERTIES: 'DBPROPERTIES';
+DEFAULT: 'DEFAULT';
 DEFINED: 'DEFINED';
 DELETE: 'DELETE';
 DELIMITED: 'DELIMITED';
diff --git a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4 b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
index 14ad9c35c4..642b65a182 100644
--- a/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
+++ b/fe/fe-core/src/main/antlr4/org/apache/doris/nereids/DorisParser.g4
@@ -45,6 +45,10 @@ statement
         (WITH LABEL labelName=identifier)? cols=identifierList?  // label and columns define
         (LEFT_BRACKET hints=identifierSeq RIGHT_BRACKET)?  // hint define
         query                                                          #insertIntoQuery
+    | explain? UPDATE tableName=multipartIdentifier tableAlias
+        SET updateAssignmentSeq
+        fromClause?
+        whereClause                                                    #update
     ;
 
 // -----------------Command accessories-----------------
@@ -184,11 +188,20 @@ hintStatement
 hintAssignment
     : key=identifierOrText (EQ (constantValue=constant | identifierValue=identifier))?
     ;
+    
+updateAssignment
+    : col=multipartIdentifier EQ (expression | DEFAULT)
+    ;
+    
+updateAssignmentSeq
+    : assignments+=updateAssignment (COMMA assignments+=updateAssignment)*
+    ;
 
 lateralView
     : LATERAL VIEW functionName=identifier LEFT_PAREN (expression (COMMA expression)*)? RIGHT_PAREN
       tableName=identifier AS columnName=identifier
     ;
+
 queryOrganization
     : sortClause? limitClause?
     ;
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
index 491b1e80f6..f0cc946500 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/parser/LogicalPlanBuilder.java
@@ -102,6 +102,9 @@ import org.apache.doris.nereids.DorisParser.TvfPropertyContext;
 import org.apache.doris.nereids.DorisParser.TvfPropertyItemContext;
 import org.apache.doris.nereids.DorisParser.TypeConstructorContext;
 import org.apache.doris.nereids.DorisParser.UnitIdentifierContext;
+import org.apache.doris.nereids.DorisParser.UpdateAssignmentContext;
+import org.apache.doris.nereids.DorisParser.UpdateAssignmentSeqContext;
+import org.apache.doris.nereids.DorisParser.UpdateContext;
 import org.apache.doris.nereids.DorisParser.UserIdentifyContext;
 import org.apache.doris.nereids.DorisParser.UserVariableContext;
 import org.apache.doris.nereids.DorisParser.WhereClauseContext;
@@ -210,6 +213,7 @@ import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand;
 import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
 import org.apache.doris.nereids.trees.plans.commands.ExplainCommand.ExplainLevel;
 import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.UpdateCommand;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
 import org.apache.doris.nereids.trees.plans.logical.LogicalCTE;
 import org.apache.doris.nereids.trees.plans.logical.LogicalCheckPolicy;
@@ -315,6 +319,23 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
         return new InsertIntoTableCommand(sink, labelName);
     }
 
+    @Override
+    public LogicalPlan visitUpdate(UpdateContext ctx) {
+        LogicalPlan query = withCheckPolicy(new UnboundRelation(
+                RelationUtil.newRelationId(), visitMultipartIdentifier(ctx.tableName)));
+        query = withTableAlias(query, ctx.tableAlias());
+        if (ctx.fromClause() != null) {
+            query = withRelations(query, ctx.fromClause());
+        }
+        query = withFilter(query, Optional.of(ctx.whereClause()));
+        String tableAlias = null;
+        if (ctx.tableAlias().strictIdentifier() != null) {
+            tableAlias = ctx.tableAlias().getText();
+        }
+        return withExplain(new UpdateCommand(visitMultipartIdentifier(ctx.tableName), tableAlias,
+                visitUpdateAssignmentSeq(ctx.updateAssignmentSeq()), query), ctx.explain());
+    }
+
     /**
      * Visit multi-statements.
      */
@@ -1268,25 +1289,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
 
     @Override
     public LogicalPlan visitFromClause(FromClauseContext ctx) {
-        return ParserUtils.withOrigin(ctx, () -> {
-            LogicalPlan left = null;
-            for (RelationContext relation : ctx.relation()) {
-                // build left deep join tree
-                LogicalPlan right = visitRelation(relation);
-                left = (left == null) ? right :
-                        new LogicalJoin<>(
-                                JoinType.CROSS_JOIN,
-                                ExpressionUtils.EMPTY_CONDITION,
-                                ExpressionUtils.EMPTY_CONDITION,
-                                JoinHint.NONE,
-                                Optional.empty(),
-                                left,
-                                right);
-                left = withJoinRelations(left, relation);
-                // TODO: pivot and lateral view
-            }
-            return left;
-        });
+        return ParserUtils.withOrigin(ctx, () -> withRelations(null, ctx));
     }
 
     /* ********************************************************************************************
@@ -1318,6 +1321,19 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
             .collect(ImmutableList.toImmutableList());
     }
 
+    @Override
+    public EqualTo visitUpdateAssignment(UpdateAssignmentContext ctx) {
+        return new EqualTo(new UnboundSlot(visitMultipartIdentifier(ctx.multipartIdentifier())),
+                getExpression(ctx.expression()));
+    }
+
+    @Override
+    public List<EqualTo> visitUpdateAssignmentSeq(UpdateAssignmentSeqContext ctx) {
+        return ctx.assignments.stream()
+                .map(this::visitUpdateAssignment)
+                .collect(Collectors.toList());
+    }
+
     /**
      * get OrderKey.
      *
@@ -1626,6 +1642,26 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> {
         });
     }
 
+    private LogicalPlan withRelations(LogicalPlan inputPlan, FromClauseContext ctx) {
+        LogicalPlan left = inputPlan;
+        for (RelationContext relation : ctx.relation()) {
+            // build left deep join tree
+            LogicalPlan right = visitRelation(relation);
+            left = (left == null) ? right :
+                    new LogicalJoin<>(
+                            JoinType.CROSS_JOIN,
+                            ExpressionUtils.EMPTY_CONDITION,
+                            ExpressionUtils.EMPTY_CONDITION,
+                            JoinHint.NONE,
+                            Optional.empty(),
+                            left,
+                            right);
+            left = withJoinRelations(left, relation);
+            // TODO: pivot and lateral view
+        }
+        return left;
+    }
+
     private LogicalPlan withFilter(LogicalPlan input, Optional<WhereClauseContext> whereCtx) {
         return input.optionalMap(whereCtx, () ->
             new LogicalFilter<>(ExpressionUtils.extractConjunctionToSet(
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Explainable.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Explainable.java
new file mode 100644
index 0000000000..46771392e5
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/Explainable.java
@@ -0,0 +1,27 @@
+// 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.qe.ConnectContext;
+
+/**
+ * plan can be explained.
+ */
+public interface Explainable {
+    Plan getExplainPlan(ConnectContext ctx) throws Exception;
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java
index a312c1738c..820e3861d8 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/ExplainCommand.java
@@ -18,8 +18,10 @@
 package org.apache.doris.nereids.trees.plans.commands;
 
 import org.apache.doris.analysis.ExplainOptions;
+import org.apache.doris.common.AnalysisException;
 import org.apache.doris.nereids.NereidsPlanner;
 import org.apache.doris.nereids.glue.LogicalPlanAdapter;
+import org.apache.doris.nereids.trees.plans.Explainable;
 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;
@@ -65,7 +67,12 @@ public class ExplainCommand extends Command implements NoForward {
 
     @Override
     public void run(ConnectContext ctx, StmtExecutor executor) throws Exception {
-        LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(logicalPlan, ctx.getStatementContext());
+        LogicalPlan explainPlan = null;
+        if (!(logicalPlan instanceof Explainable)) {
+            throw new AnalysisException("explain a plan cannot be explained");
+        }
+        explainPlan = ((LogicalPlan) ((Explainable) logicalPlan).getExplainPlan(ctx));
+        LogicalPlanAdapter logicalPlanAdapter = new LogicalPlanAdapter(explainPlan, ctx.getStatementContext());
         logicalPlanAdapter.setIsExplain(new ExplainOptions(level));
         executor.setParsedStmt(logicalPlanAdapter);
         NereidsPlanner planner = new NereidsPlanner(ctx.getStatementContext());
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java
index 291cf1687c..8e7c688597 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/InsertIntoTableCommand.java
@@ -24,6 +24,8 @@ import org.apache.doris.nereids.NereidsPlanner;
 import org.apache.doris.nereids.exceptions.AnalysisException;
 import org.apache.doris.nereids.glue.LogicalPlanAdapter;
 import org.apache.doris.nereids.trees.TreeNode;
+import org.apache.doris.nereids.trees.plans.Explainable;
+import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.PlanType;
 import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
 import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapTableSink;
@@ -40,8 +42,10 @@ import com.google.common.base.Strings;
 import org.apache.logging.log4j.LogManager;
 import org.apache.logging.log4j.Logger;
 
+import java.util.Objects;
 import java.util.Optional;
 import java.util.Set;
+import javax.annotation.Nullable;
 
 /**
  * insert into select command implementation
@@ -51,22 +55,22 @@ import java.util.Set;
  *  InsertIntoTableCommand(Query())
  *  ExplainCommand(Query())
  */
-public class InsertIntoTableCommand extends Command implements ForwardWithSync {
+public class InsertIntoTableCommand extends Command implements ForwardWithSync, Explainable {
 
     public static final Logger LOG = LogManager.getLogger(InsertIntoTableCommand.class);
 
     private final LogicalPlan logicalQuery;
-    private final String labelName;
+    private final @Nullable String labelName;
     private NereidsPlanner planner;
     private boolean isTxnBegin = false;
 
     /**
      * constructor
      */
-    public InsertIntoTableCommand(LogicalPlan logicalQuery, String labelName) {
+    public InsertIntoTableCommand(LogicalPlan logicalQuery, @Nullable String labelName) {
         super(PlanType.INSERT_INTO_TABLE_COMMAND);
-        Preconditions.checkNotNull(logicalQuery, "logicalQuery cannot be null in InsertIntoTableCommand");
-        this.logicalQuery = logicalQuery;
+        this.logicalQuery = Objects.requireNonNull(logicalQuery,
+                "logicalQuery cannot be null in InsertIntoTableCommand");
         this.labelName = labelName;
     }
 
@@ -150,6 +154,11 @@ public class InsertIntoTableCommand extends Command implements ForwardWithSync {
         }
     }
 
+    @Override
+    public Plan getExplainPlan(ConnectContext ctx) {
+        return this.logicalQuery;
+    }
+
     @Override
     public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
         return visitor.visitInsertIntoCommand(this, context);
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java
new file mode 100644
index 0000000000..a582c22394
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/UpdateCommand.java
@@ -0,0 +1,156 @@
+// 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.catalog.Column;
+import org.apache.doris.catalog.KeysType;
+import org.apache.doris.catalog.OlapTable;
+import org.apache.doris.catalog.Table;
+import org.apache.doris.catalog.TableIf;
+import org.apache.doris.common.AnalysisException;
+import org.apache.doris.nereids.analyzer.UnboundOlapTableSink;
+import org.apache.doris.nereids.analyzer.UnboundSlot;
+import org.apache.doris.nereids.trees.expressions.Alias;
+import org.apache.doris.nereids.trees.expressions.EqualTo;
+import org.apache.doris.nereids.trees.expressions.Expression;
+import org.apache.doris.nereids.trees.expressions.NamedExpression;
+import org.apache.doris.nereids.trees.plans.Explainable;
+import org.apache.doris.nereids.trees.plans.Plan;
+import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.trees.plans.logical.LogicalProject;
+import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor;
+import org.apache.doris.nereids.util.RelationUtil;
+import org.apache.doris.qe.ConnectContext;
+import org.apache.doris.qe.SessionVariable;
+import org.apache.doris.qe.StmtExecutor;
+
+import com.google.common.collect.ImmutableList;
+import com.google.common.collect.Lists;
+import com.google.common.collect.Maps;
+
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import javax.annotation.Nullable;
+
+/**
+ * update command
+ * the two case will be handled as:
+ * case 1:
+ *  update table t1 set v1 = v1 + 1 where k1 = 1 and k2 = 2;
+ * =>
+ *  insert into table (v1) select v1 + 1 from table t1 where k1 = 1 and k2 = 2
+ * case 2:
+ *  update t1 set t1.c1 = t2.c1, t1.c3 = t2.c3 * 100
+ *  from t2 inner join t3 on t2.id = t3.id
+ *  where t1.id = t2.id;
+ * =>
+ *  insert into t1 (c1, c3) select t2.c1, t2.c3 * 100 from t1 join t2 inner join t3 on t2.id = t3.id where t1.id = t2.id
+ */
+public class UpdateCommand extends Command implements ForwardWithSync, Explainable {
+    private final List<EqualTo> assignments;
+    private final List<String> nameParts;
+    private final @Nullable String tableAlias;
+    private final LogicalPlan logicalQuery;
+    private OlapTable targetTable;
+
+    /**
+     * constructor
+     */
+    public UpdateCommand(List<String> nameParts, @Nullable String tableAlias, List<EqualTo> assignments,
+            LogicalPlan logicalQuery) {
+        super(PlanType.UPDATE_COMMAND);
+        this.nameParts = ImmutableList.copyOf(Objects.requireNonNull(nameParts,
+                "tableName is required in update command"));
+        this.assignments = ImmutableList.copyOf(Objects.requireNonNull(assignments,
+                "assignment is required in update command"));
+        this.tableAlias = tableAlias;
+        this.logicalQuery = Objects.requireNonNull(logicalQuery, "logicalQuery is required in update command");
+    }
+
+    @Override
+    public void run(ConnectContext ctx, StmtExecutor executor) throws Exception {
+        new InsertIntoTableCommand(completeQueryPlan(ctx, logicalQuery), null).run(ctx, executor);
+    }
+
+    /**
+     * add LogicalOlapTableSink node, public for test.
+     */
+    public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuery) throws AnalysisException {
+        checkTable(ctx);
+
+        Map<String, Expression> colNameToExpression = Maps.newHashMap();
+        for (EqualTo equalTo : assignments) {
+            List<String> nameParts = ((UnboundSlot) equalTo.left()).getNameParts();
+            colNameToExpression.put(nameParts.get(nameParts.size() - 1), equalTo.right());
+        }
+        List<NamedExpression> selectItems = Lists.newArrayList();
+        String tableName = tableAlias != null ? tableAlias : targetTable.getName();
+        for (Column column : targetTable.getFullSchema()) {
+            if (!column.isVisible()) {
+                continue;
+            }
+            if (colNameToExpression.containsKey(column.getName())) {
+                Expression expr = colNameToExpression.get(column.getName());
+                selectItems.add(expr instanceof UnboundSlot
+                        ? ((NamedExpression) expr)
+                        : new Alias(expr, expr.toSql()));
+            } else {
+                selectItems.add(new UnboundSlot(tableName, column.getName()));
+            }
+        }
+
+        logicalQuery = new LogicalProject<>(selectItems, logicalQuery);
+
+        // make UnboundTableSink
+        return new UnboundOlapTableSink<>(nameParts, null, null, null, logicalQuery);
+    }
+
+    private void checkTable(ConnectContext ctx) throws AnalysisException {
+        if (ctx.getSessionVariable().isInDebugMode()) {
+            throw new AnalysisException("Update is forbidden since current session is in debug mode."
+                    + " Please check the following session variables: "
+                    + String.join(", ", SessionVariable.DEBUG_VARIABLES));
+        }
+        List<String> tableQualifier = RelationUtil.getQualifierName(ctx, nameParts);
+        TableIf table = RelationUtil.getTable(tableQualifier, ctx.getEnv());
+        if (!(table instanceof OlapTable)) {
+            throw new AnalysisException("target table in update command should be an olapTable");
+        }
+        targetTable = ((OlapTable) table);
+        if (targetTable.getType() != Table.TableType.OLAP
+                || targetTable.getKeysType() != KeysType.UNIQUE_KEYS) {
+            throw new AnalysisException("Only unique table could be updated.");
+        }
+    }
+
+    @Override
+    public Plan getExplainPlan(ConnectContext ctx) throws AnalysisException {
+        return completeQueryPlan(ctx, logicalQuery);
+    }
+
+    public LogicalPlan getLogicalQuery() {
+        return logicalQuery;
+    }
+
+    @Override
+    public <R, C> R accept(PlanVisitor<R, C> visitor, C context) {
+        return visitor.visitUpdateCommand(this, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java
index 15ea90e909..d8fb322073 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/AbstractLogicalPlan.java
@@ -21,8 +21,10 @@ import org.apache.doris.nereids.memo.GroupExpression;
 import org.apache.doris.nereids.properties.LogicalProperties;
 import org.apache.doris.nereids.properties.UnboundLogicalProperties;
 import org.apache.doris.nereids.trees.plans.AbstractPlan;
+import org.apache.doris.nereids.trees.plans.Explainable;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.qe.ConnectContext;
 
 import java.util.Optional;
 import java.util.function.Supplier;
@@ -30,7 +32,7 @@ import java.util.function.Supplier;
 /**
  * Abstract class for all concrete logical plan.
  */
-public abstract class AbstractLogicalPlan extends AbstractPlan implements LogicalPlan {
+public abstract class AbstractLogicalPlan extends AbstractPlan implements LogicalPlan, Explainable {
 
     private Supplier<Boolean> hasUnboundExpressions = () -> super.hasUnboundExpression();
 
@@ -63,4 +65,9 @@ public abstract class AbstractLogicalPlan extends AbstractPlan implements Logica
             return new LogicalProperties(this::computeOutput);
         }
     }
+
+    @Override
+    public Plan getExplainPlan(ConnectContext ctx) {
+        return this;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java
index 2cdab8c32b..14b989f414 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/AbstractPhysicalPlan.java
@@ -21,8 +21,10 @@ import org.apache.doris.nereids.memo.GroupExpression;
 import org.apache.doris.nereids.properties.LogicalProperties;
 import org.apache.doris.nereids.properties.PhysicalProperties;
 import org.apache.doris.nereids.trees.plans.AbstractPlan;
+import org.apache.doris.nereids.trees.plans.Explainable;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.PlanType;
+import org.apache.doris.qe.ConnectContext;
 import org.apache.doris.statistics.Statistics;
 
 import java.util.Optional;
@@ -31,7 +33,7 @@ import javax.annotation.Nullable;
 /**
  * Abstract class for all concrete physical plan.
  */
-public abstract class AbstractPhysicalPlan extends AbstractPlan implements PhysicalPlan {
+public abstract class AbstractPhysicalPlan extends AbstractPlan implements PhysicalPlan, Explainable {
 
     protected final PhysicalProperties physicalProperties;
 
@@ -54,4 +56,9 @@ public abstract class AbstractPhysicalPlan extends AbstractPlan implements Physi
     public PhysicalProperties getPhysicalProperties() {
         return physicalProperties;
     }
+
+    @Override
+    public Plan getExplainPlan(ConnectContext ctx) {
+        return this;
+    }
 }
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java
new file mode 100644
index 0000000000..5adab2cc96
--- /dev/null
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/CommandVisitor.java
@@ -0,0 +1,46 @@
+// 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.visitor;
+
+import org.apache.doris.nereids.trees.plans.commands.Command;
+import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand;
+import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
+import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand;
+import org.apache.doris.nereids.trees.plans.commands.UpdateCommand;
+
+/** CommandVisitor. */
+public interface CommandVisitor<R, C> {
+    R visitCommand(Command command, C context);
+
+    default R visitExplainCommand(ExplainCommand explain, C context) {
+        return visitCommand(explain, context);
+    }
+
+    default R visitCreatePolicyCommand(CreatePolicyCommand createPolicy, C context) {
+        return visitCommand(createPolicy, context);
+    }
+
+    default R visitInsertIntoCommand(InsertIntoTableCommand insertIntoSelectCommand,
+            C context) {
+        return visitCommand(insertIntoSelectCommand, context);
+    }
+
+    default R visitUpdateCommand(UpdateCommand updateCommand, C context) {
+        return visitCommand(updateCommand, context);
+    }
+}
diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
index 4b9e480453..799988c449 100644
--- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
+++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/visitor/PlanVisitor.java
@@ -24,9 +24,6 @@ import org.apache.doris.nereids.analyzer.UnboundTVFRelation;
 import org.apache.doris.nereids.trees.plans.GroupPlan;
 import org.apache.doris.nereids.trees.plans.Plan;
 import org.apache.doris.nereids.trees.plans.commands.Command;
-import org.apache.doris.nereids.trees.plans.commands.CreatePolicyCommand;
-import org.apache.doris.nereids.trees.plans.commands.ExplainCommand;
-import org.apache.doris.nereids.trees.plans.commands.InsertIntoTableCommand;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAggregate;
 import org.apache.doris.nereids.trees.plans.logical.LogicalApply;
 import org.apache.doris.nereids.trees.plans.logical.LogicalAssertNumRows;
@@ -103,7 +100,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalWindow;
  * @param <R> Return type of each visit method.
  * @param <C> Context type.
  */
-public abstract class PlanVisitor<R, C> {
+public abstract class PlanVisitor<R, C> implements CommandVisitor<R, C> {
 
     public abstract R visit(Plan plan, C context);
 
@@ -115,19 +112,6 @@ public abstract class PlanVisitor<R, C> {
         return visit(command, context);
     }
 
-    public R visitExplainCommand(ExplainCommand explain, C context) {
-        return visitCommand(explain, context);
-    }
-
-    public R visitCreatePolicyCommand(CreatePolicyCommand createPolicy, C context) {
-        return visitCommand(createPolicy, context);
-    }
-
-    public R visitInsertIntoCommand(InsertIntoTableCommand insertIntoSelectCommand,
-            C context) {
-        return visit(insertIntoSelectCommand, context);
-    }
-
     // *******************************
     // Logical plans
     // *******************************
diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/UpdateCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/UpdateCommandTest.java
new file mode 100644
index 0000000000..440d5345f6
--- /dev/null
+++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/UpdateCommandTest.java
@@ -0,0 +1,126 @@
+// 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.common.AnalysisException;
+import org.apache.doris.nereids.parser.NereidsParser;
+import org.apache.doris.nereids.trees.plans.commands.UpdateCommand;
+import org.apache.doris.nereids.trees.plans.logical.LogicalPlan;
+import org.apache.doris.nereids.util.PlanChecker;
+import org.apache.doris.nereids.util.PlanPatternMatchSupported;
+import org.apache.doris.utframe.TestWithFeService;
+
+import org.junit.jupiter.api.Assertions;
+import org.junit.jupiter.api.Test;
+
+public class UpdateCommandTest extends TestWithFeService implements PlanPatternMatchSupported {
+    @Override
+    public void runBeforeAll() throws Exception {
+        createDatabase("test");
+        connectContext.setDatabase("default_cluster:test");
+        createTable("create table t1 (\n"
+                + "    k1 int,\n"
+                + "    k2 int,\n"
+                + "    v1 int,\n"
+                + "    v2 int\n"
+                + ")\n"
+                + "unique key(k1, k2)\n"
+                + "distributed by hash(k1) buckets 4\n"
+                + "properties(\n"
+                + "    \"replication_num\"=\"1\"\n"
+                + ")");
+        createTable("create table t2 (\n"
+                + "    k1 int,\n"
+                + "    k2 int,\n"
+                + "    v1 int,\n"
+                + "    v2 int\n"
+                + ")\n"
+                + "unique key(k1, k2)\n"
+                + "distributed by hash(k1) buckets 4\n"
+                + "properties(\n"
+                + "    \"replication_num\"=\"1\"\n"
+                + ")");
+        createTable("create table src (\n"
+                + "    k1 int,\n"
+                + "    k2 int,\n"
+                + "    v1 int,\n"
+                + "    v2 int\n"
+                + ")\n"
+                + "duplicate key(k1, k2)\n"
+                + "distributed by hash(k1) buckets 4\n"
+                + "properties(\n"
+                + "    \"replication_num\"=\"1\"\n"
+                + ")");
+    }
+
+    @Test
+    public void testSimpleUpdate() throws AnalysisException {
+        String sql = "update t1 set v1 = v1 + 2, v2 = v1 * 2 where k1 = 3";
+        LogicalPlan parsed = new NereidsParser().parseSingle(sql);
+        Assertions.assertTrue(parsed instanceof UpdateCommand);
+        UpdateCommand command = ((UpdateCommand) parsed);
+        LogicalPlan plan = command.completeQueryPlan(connectContext, command.getLogicalQuery());
+        PlanChecker.from(connectContext, plan)
+                .analyze(plan)
+                .rewrite()
+                .matches(
+                        logicalOlapTableSink(
+                                logicalProject(
+                                        logicalFilter(
+                                                logicalOlapScan()
+                                        )
+                                )
+                        )
+                );
+    }
+
+    @Test
+    public void testFromClauseUpdate() throws AnalysisException {
+        String sql = "update t1 a set v1 = t2.v1 + 2, v2 = a.v1 * 2 "
+                + "from src join t2 on src.k1 = t2.k1 where t2.k1 = a.k1";
+        LogicalPlan parsed = new NereidsParser().parseSingle(sql);
+        Assertions.assertTrue(parsed instanceof UpdateCommand);
+        UpdateCommand command = ((UpdateCommand) parsed);
+        LogicalPlan plan = command.completeQueryPlan(connectContext, command.getLogicalQuery());
+        PlanChecker.from(connectContext, plan)
+                .analyze(plan)
+                .rewrite()
+                .matches(
+                        logicalOlapTableSink(
+                                logicalProject(
+                                        logicalJoin(
+                                                logicalJoin(
+                                                        logicalProject(
+                                                                logicalFilter(
+                                                                        logicalOlapScan()
+                                                                )
+                                                        ),
+                                                        logicalProject(
+                                                                logicalOlapScan())
+                                                ),
+                                                logicalProject(
+                                                        logicalFilter(
+                                                                logicalOlapScan()
+                                                        )
+                                                )
+                                        )
+                                )
+                        )
+                );
+    }
+}
diff --git a/regression-test/data/nereids_p0/update/update_unique_table.out b/regression-test/data/nereids_p0/update/update_unique_table.out
index 9c04fa150f..24a7d0462f 100644
--- a/regression-test/data/nereids_p0/update/update_unique_table.out
+++ b/regression-test/data/nereids_p0/update/update_unique_table.out
@@ -2,15 +2,15 @@
 -- !sql --
 1	10	1	1	1.0	2000-01-01
 2	20	2	2	2.0	2000-01-02
-3	30	3	3	3.0	2000-01-03
+3	30	5	3	3.0	2000-01-03
 
 -- !sql --
 1	10	2	1	2.0	2000-01-01
 2	20	2	2	2.0	2000-01-02
-3	30	3	3	3.0	2000-01-03
+3	30	5	3	3.0	2000-01-03
 
 -- !sql --
 1	10	10	1	1000.0	2000-01-01
 2	20	2	2	2.0	2000-01-02
-3	30	3	3	3.0	2000-01-03
+3	30	5	3	3.0	2000-01-03
 
diff --git a/regression-test/suites/nereids_p0/update/load.groovy b/regression-test/suites/nereids_p0/update/load.groovy
new file mode 100644
index 0000000000..16f7ba5a07
--- /dev/null
+++ b/regression-test/suites/nereids_p0/update/load.groovy
@@ -0,0 +1,81 @@
+// 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.
+
+suite("load") {
+    sql '''
+        create table t1 (
+            id int,
+            id1 int,
+            c1 bigint,
+            c2 string,
+            c3 double,
+            c4 date
+        ) unique key (id, id1)
+        distributed by hash(id, id1)
+        properties(
+            'replication_num'='1',
+            "function_column.sequence_col" = "c4"
+        );
+    '''
+
+    sql '''
+        create table t2 (
+            id int,
+            c1 bigint,
+            c2 string,
+            c3 double,
+            c4 date
+        ) unique key (id)
+        distributed by hash(id)
+        properties(
+            'replication_num'='1'
+        );
+    '''
+
+    sql '''
+        create table t3 (
+            id int
+        ) distributed by hash(id)
+        properties(
+            'replication_num'='1'
+        );
+    '''
+
+    sql '''
+        INSERT INTO t1 VALUES
+            (1, 10, 1, '1', 1.0, '2000-01-01'),
+            (2, 20, 2, '2', 2.0, '2000-01-02'),
+            (3, 30, 3, '3', 3.0, '2000-01-03');
+    '''
+
+    sql '''
+
+        INSERT INTO t2 VALUES
+            (1, 10, '10', 10.0, '2000-01-10'),
+            (2, 20, '20', 20.0, '2000-01-20'),
+            (3, 30, '30', 30.0, '2000-01-30'),
+            (4, 4, '4', 4.0, '2000-01-04'),
+            (5, 5, '5', 5.0, '2000-01-05');
+    '''
+
+    sql '''
+        INSERT INTO t3 VALUES
+            (1),
+            (4),
+            (5);
+    '''
+}
diff --git a/regression-test/suites/nereids_p0/update/update_unique_table.groovy b/regression-test/suites/nereids_p0/update/update_unique_table.groovy
new file mode 100644
index 0000000000..358a3bb9cf
--- /dev/null
+++ b/regression-test/suites/nereids_p0/update/update_unique_table.groovy
@@ -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.
+
+suite('update_unique_table') {
+    sql 'set enable_nereids_planner=true'
+    sql 'set enable_fallback_to_original_planner=false'
+    sql 'set enable_nereids_dml=true'
+
+    sql 'update t1 set c1 = 5 where id = 3'
+    
+    qt_sql 'select * from t1 order by id'
+
+    sql 'update t1 set c1 = c1 + 1, c3 = c2 * 2 where id = 1'
+
+    qt_sql 'select * from t1 order by id'
+
+    sql '''
+        update t1
+        set t1.c1 = t2.c1, t1.c3 = t2.c3 * 100
+        from t2 inner join t3 on t2.id = t3.id
+        where t1.id = t2.id;
+    '''
+
+    qt_sql 'select * from t1 order by id'
+}


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