This is an automated email from the ASF dual-hosted git repository. englefly pushed a commit to branch branch-2.1 in repository https://gitbox.apache.org/repos/asf/doris.git
The following commit(s) were added to refs/heads/branch-2.1 by this push: new b3f2bd20e32 [feat](nereids) support explain delete from clause #36782 (#37100) b3f2bd20e32 is described below commit b3f2bd20e3254a3021834671ee2af9ca77ba6646 Author: minghong <engle...@gmail.com> AuthorDate: Wed Jul 3 15:08:08 2024 +0800 [feat](nereids) support explain delete from clause #36782 (#37100) ## Proposed changes pick from #36782 support explain like: explain delete from T where A=1 Issue Number: close #xxx <!--Describe your changes.--> (cherry picked from commit dc369cd13096dbb90700f7fbf8f35a9059d9906f) ## Proposed changes Issue Number: close #xxx <!--Describe your changes.--> --- .../doris/nereids/parser/LogicalPlanBuilder.java | 16 ++- .../trees/plans/commands/DeleteFromCommand.java | 92 ++++++++++++++- .../plans/commands/DeleteFromUsingCommand.java | 90 +++------------ .../data/delete_p0/test_delete_on_value.out | 2 +- .../delete/delete_mow_partial_update.out | 12 +- .../partial_update/test_partial_update_delete.out | 12 +- .../delete/delete_mow_partial_update.groovy | 2 +- .../suites/nereids_p0/explain/explain_dml.groovy | 124 +++++++++++++++++++++ 8 files changed, 252 insertions(+), 98 deletions(-) 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 27d0e4cbd3c..543a5630243 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 @@ -925,9 +925,12 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { if (ctx.tableAlias().strictIdentifier() != null) { tableAlias = ctx.tableAlias().getText(); } - if (ctx.USING() == null && ctx.cte() == null && ctx.explain() == null) { + + Command deleteCommand; + if (ctx.USING() == null && ctx.cte() == null) { query = withFilter(query, Optional.ofNullable(ctx.whereClause())); - return new DeleteFromCommand(tableName, tableAlias, partitionSpec.first, partitionSpec.second, query); + deleteCommand = new DeleteFromCommand(tableName, tableAlias, partitionSpec.first, + partitionSpec.second, query); } else { // convert to insert into select query = withRelations(query, ctx.relations().relation()); @@ -936,8 +939,13 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { if (ctx.cte() != null) { cte = Optional.ofNullable(withCte(query, ctx.cte())); } - return withExplain(new DeleteFromUsingCommand(tableName, tableAlias, - partitionSpec.first, partitionSpec.second, query, cte), ctx.explain()); + deleteCommand = new DeleteFromUsingCommand(tableName, tableAlias, + partitionSpec.first, partitionSpec.second, query, cte); + } + if (ctx.explain() != null) { + return withExplain(deleteCommand, ctx.explain()); + } else { + return deleteCommand; } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java index 6563d815382..c485802ae2e 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromCommand.java @@ -28,11 +28,15 @@ import org.apache.doris.catalog.Env; import org.apache.doris.catalog.KeysType; import org.apache.doris.catalog.MaterializedIndexMeta; import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.TableIf; import org.apache.doris.common.Config; import org.apache.doris.common.ErrorCode; import org.apache.doris.mysql.privilege.PrivPredicate; import org.apache.doris.nereids.NereidsPlanner; +import org.apache.doris.nereids.analyzer.UnboundAlias; import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.analyzer.UnboundTableSinkCreator; import org.apache.doris.nereids.exceptions.AnalysisException; import org.apache.doris.nereids.glue.LogicalPlanAdapter; import org.apache.doris.nereids.rules.RuleType; @@ -41,12 +45,17 @@ import org.apache.doris.nereids.trees.expressions.ComparisonPredicate; import org.apache.doris.nereids.trees.expressions.Expression; import org.apache.doris.nereids.trees.expressions.InPredicate; import org.apache.doris.nereids.trees.expressions.IsNull; +import org.apache.doris.nereids.trees.expressions.NamedExpression; import org.apache.doris.nereids.trees.expressions.Not; import org.apache.doris.nereids.trees.expressions.SlotReference; import org.apache.doris.nereids.trees.expressions.literal.Literal; +import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; +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.commands.info.DMLCommandType; 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.physical.PhysicalDistribute; import org.apache.doris.nereids.trees.plans.physical.PhysicalEmptyRelation; import org.apache.doris.nereids.trees.plans.physical.PhysicalFilter; @@ -54,6 +63,7 @@ import org.apache.doris.nereids.trees.plans.physical.PhysicalOlapScan; import org.apache.doris.nereids.trees.plans.physical.PhysicalProject; import org.apache.doris.nereids.trees.plans.physical.PhysicalUnary; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.RelationUtil; import org.apache.doris.nereids.util.Utils; import org.apache.doris.qe.ConnectContext; import org.apache.doris.qe.SessionVariable; @@ -61,6 +71,7 @@ import org.apache.doris.qe.StmtExecutor; import org.apache.doris.qe.VariableMgr; import com.google.common.base.Preconditions; +import com.google.common.collect.ImmutableList; import com.google.common.collect.Lists; import org.apache.commons.lang3.StringUtils; @@ -73,13 +84,13 @@ import java.util.stream.Collectors; /** * delete from unique key table. */ -public class DeleteFromCommand extends Command implements ForwardWithSync { +public class DeleteFromCommand extends Command implements ForwardWithSync, Explainable { - private final List<String> nameParts; - private final String tableAlias; - private final boolean isTempPart; - private final List<String> partitions; - private final LogicalPlan logicalQuery; + protected final List<String> nameParts; + protected final String tableAlias; + protected final boolean isTempPart; + protected final List<String> partitions; + protected final LogicalPlan logicalQuery; /** * constructor @@ -347,4 +358,73 @@ public class DeleteFromCommand extends Command implements ForwardWithSync { public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { return visitor.visitDeleteFromCommand(this, context); } + + @Override + public Plan getExplainPlan(ConnectContext ctx) { + if (!ctx.getSessionVariable().isEnableNereidsDML()) { + try { + ctx.getSessionVariable().enableFallbackToOriginalPlannerOnce(); + } catch (Exception e) { + throw new AnalysisException("failed to set fallback to original planner to true", e); + } + throw new AnalysisException("Nereids DML is disabled, will try to fall back to the original planner"); + } + return completeQueryPlan(ctx, logicalQuery); + } + + private OlapTable getTargetTable(ConnectContext ctx) { + List<String> qualifiedTableName = RelationUtil.getQualifierName(ctx, nameParts); + TableIf table = RelationUtil.getTable(qualifiedTableName, ctx.getEnv()); + if (!(table instanceof OlapTable)) { + throw new AnalysisException("table must be olapTable in delete command"); + } + return ((OlapTable) table); + } + + /** + * for explain command + */ + public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuery) { + OlapTable targetTable = getTargetTable(ctx); + checkTargetTable(targetTable); + // add select and insert node. + List<NamedExpression> selectLists = Lists.newArrayList(); + List<String> cols = Lists.newArrayList(); + boolean isMow = targetTable.getEnableUniqueKeyMergeOnWrite(); + String tableName = tableAlias != null ? tableAlias : targetTable.getName(); + for (Column column : targetTable.getFullSchema()) { + if (column.getName().equalsIgnoreCase(Column.DELETE_SIGN)) { + selectLists.add(new UnboundAlias(new TinyIntLiteral(((byte) 1)), Column.DELETE_SIGN)); + } else if (column.getName().equalsIgnoreCase(Column.SEQUENCE_COL) + && targetTable.getSequenceMapCol() != null) { + selectLists.add(new UnboundSlot(tableName, targetTable.getSequenceMapCol())); + } else if (column.isKey()) { + selectLists.add(new UnboundSlot(tableName, column.getName())); + } else if (!isMow && (!column.isVisible() || (!column.isAllowNull() && !column.hasDefaultValue()))) { + selectLists.add(new UnboundSlot(tableName, column.getName())); + } else { + selectLists.add(new UnboundSlot(tableName, column.getName())); + } + cols.add(column.getName()); + } + + logicalQuery = new LogicalProject<>(selectLists, logicalQuery); + + boolean isPartialUpdate = targetTable.getEnableUniqueKeyMergeOnWrite() + && cols.size() < targetTable.getColumns().size(); + logicalQuery = handleCte(logicalQuery); + // make UnboundTableSink + return UnboundTableSinkCreator.createUnboundTableSink(nameParts, cols, ImmutableList.of(), + isTempPart, partitions, isPartialUpdate, DMLCommandType.DELETE, logicalQuery); + } + + protected LogicalPlan handleCte(LogicalPlan logicalPlan) { + return logicalPlan; + } + + protected void checkTargetTable(OlapTable targetTable) { + if (targetTable.getKeysType() != KeysType.UNIQUE_KEYS) { + throw new AnalysisException("delete command on aggregate/duplicate table is not explainable"); + } + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromUsingCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromUsingCommand.java index ff70c75558d..e452f6f1cb1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromUsingCommand.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteFromUsingCommand.java @@ -17,57 +17,32 @@ 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.nereids.analyzer.UnboundAlias; -import org.apache.doris.nereids.analyzer.UnboundSlot; -import org.apache.doris.nereids.analyzer.UnboundTableSinkCreator; import org.apache.doris.nereids.exceptions.AnalysisException; -import org.apache.doris.nereids.trees.expressions.NamedExpression; -import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; -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.commands.info.DMLCommandType; import org.apache.doris.nereids.trees.plans.commands.insert.InsertIntoTableCommand; 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.Utils; 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 java.util.List; import java.util.Optional; /** * delete from unique key table. */ -public class DeleteFromUsingCommand extends Command implements ForwardWithSync, Explainable { - - private final List<String> nameParts; - private final String tableAlias; - private final boolean isTempPart; - private final List<String> partitions; +public class DeleteFromUsingCommand extends DeleteFromCommand { private final Optional<LogicalPlan> cte; - private final LogicalPlan logicalQuery; /** * constructor */ public DeleteFromUsingCommand(List<String> nameParts, String tableAlias, boolean isTempPart, List<String> partitions, LogicalPlan logicalQuery, Optional<LogicalPlan> cte) { - super(PlanType.DELETE_COMMAND); - this.nameParts = Utils.copyRequiredList(nameParts); - this.tableAlias = tableAlias; - this.isTempPart = isTempPart; - this.partitions = Utils.copyRequiredList(partitions); + super(nameParts, tableAlias, isTempPart, partitions, logicalQuery); this.cte = cte; - this.logicalQuery = logicalQuery; } @Override @@ -81,63 +56,30 @@ public class DeleteFromUsingCommand extends Command implements ForwardWithSync, executor); } - /** - * public for test - */ - public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuery) { - OlapTable targetTable = CommandUtils.checkAndGetDeleteTargetTable(ctx, nameParts); - // add select and insert node. - List<NamedExpression> selectLists = Lists.newArrayList(); - List<String> cols = Lists.newArrayList(); - boolean isMow = targetTable.getEnableUniqueKeyMergeOnWrite(); - String tableName = tableAlias != null ? tableAlias : targetTable.getName(); - for (Column column : targetTable.getFullSchema()) { - if (column.getName().equalsIgnoreCase(Column.DELETE_SIGN)) { - selectLists.add(new UnboundAlias(new TinyIntLiteral(((byte) 1)), Column.DELETE_SIGN)); - } else if (column.getName().equalsIgnoreCase(Column.SEQUENCE_COL)) { - selectLists.add(new UnboundSlot(tableName, targetTable.getSequenceMapCol())); - } else if (column.isKey()) { - selectLists.add(new UnboundSlot(tableName, column.getName())); - } else if (!isMow && (!column.isVisible() || (!column.isAllowNull() && !column.hasDefaultValue()))) { - selectLists.add(new UnboundSlot(tableName, column.getName())); - } else { - continue; - } - cols.add(column.getName()); - } - - logicalQuery = new LogicalProject<>(selectLists, logicalQuery); + @Override + protected LogicalPlan handleCte(LogicalPlan logicalPlan) { if (cte.isPresent()) { - logicalQuery = ((LogicalPlan) cte.get().withChildren(logicalQuery)); + logicalPlan = ((LogicalPlan) cte.get().withChildren(logicalPlan)); } - - boolean isPartialUpdate = targetTable.getEnableUniqueKeyMergeOnWrite() - && cols.size() < targetTable.getColumns().size(); - - // make UnboundTableSink - return UnboundTableSinkCreator.createUnboundTableSink(nameParts, cols, ImmutableList.of(), - isTempPart, partitions, isPartialUpdate, DMLCommandType.DELETE, logicalQuery); + return logicalPlan; } + /** + * for test + */ public LogicalPlan getLogicalQuery() { return logicalQuery; } @Override - public Plan getExplainPlan(ConnectContext ctx) { - if (!ctx.getSessionVariable().isEnableNereidsDML()) { - try { - ctx.getSessionVariable().enableFallbackToOriginalPlannerOnce(); - } catch (Exception e) { - throw new AnalysisException("failed to set fallback to original planner to true", e); - } - throw new AnalysisException("Nereids DML is disabled, will try to fall back to the original planner"); - } - return completeQueryPlan(ctx, logicalQuery); + public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { + return visitor.visitDeleteFromUsingCommand(this, context); } @Override - public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { - return visitor.visitDeleteFromUsingCommand(this, context); + protected void checkTargetTable(OlapTable targetTable) { + if (targetTable.getKeysType() != KeysType.UNIQUE_KEYS) { + throw new AnalysisException("delete command on with using clause only supports unique key model"); + } } } diff --git a/regression-test/data/delete_p0/test_delete_on_value.out b/regression-test/data/delete_p0/test_delete_on_value.out index 2bc8846297d..2c8a84eebe5 100644 --- a/regression-test/data/delete_p0/test_delete_on_value.out +++ b/regression-test/data/delete_p0/test_delete_on_value.out @@ -88,7 +88,7 @@ -- !sql -- -- !sql -- -1 \N \N 1 4 10 1 1 5 0 3 5 1 1 10 0 2 10 +1 1 10 1 4 10 diff --git a/regression-test/data/nereids_p0/delete/delete_mow_partial_update.out b/regression-test/data/nereids_p0/delete/delete_mow_partial_update.out index 787f854bea2..b4237a03803 100644 --- a/regression-test/data/nereids_p0/delete/delete_mow_partial_update.out +++ b/regression-test/data/nereids_p0/delete/delete_mow_partial_update.out @@ -15,12 +15,12 @@ 5 5 -- !sql -- -1 \N 1 1 1 0 -2 \N 1 +1 1 1 2 2 0 -3 \N 1 +2 2 1 3 3 0 +3 3 1 4 4 0 5 5 0 @@ -61,12 +61,12 @@ 5 5 -- !sql -- -1 \N 1 1 1 0 -2 \N 1 +1 1 1 2 2 0 -3 \N 1 +2 2 1 3 3 0 +3 3 1 4 4 0 5 5 0 diff --git a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete.out b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete.out index 0863afd7931..c16e954d733 100644 --- a/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete.out +++ b/regression-test/data/unique_with_mow_p0/partial_update/test_partial_update_delete.out @@ -11,12 +11,12 @@ 5 5 5 5 5 -- !with_delete_sign -- -1 \N \N 0 \N 1 1 1 1 1 1 0 -2 \N \N 0 \N 1 +1 1 1 1 1 1 2 2 2 2 2 0 -3 \N \N 0 \N 1 +2 2 2 2 2 1 3 3 3 3 3 0 +3 3 3 3 3 1 4 4 4 4 4 0 5 5 5 5 5 0 @@ -53,12 +53,12 @@ 5 5 5 5 5 -- !with_delete_sign -- -1 \N \N 0 \N 1 1 1 1 1 1 0 -2 \N \N 0 \N 1 +1 1 1 1 1 1 2 2 2 2 2 0 -3 \N \N 0 \N 1 +2 2 2 2 2 1 3 3 3 3 3 0 +3 3 3 3 3 1 4 4 4 4 4 0 5 5 5 5 5 0 diff --git a/regression-test/suites/nereids_p0/delete/delete_mow_partial_update.groovy b/regression-test/suites/nereids_p0/delete/delete_mow_partial_update.groovy index bfb27ce14ba..5849e5d80ef 100644 --- a/regression-test/suites/nereids_p0/delete/delete_mow_partial_update.groovy +++ b/regression-test/suites/nereids_p0/delete/delete_mow_partial_update.groovy @@ -15,7 +15,7 @@ // specific language governing permissions and limitations // under the License. -suite('nereids_delete_mow_partial_update') { +suite('delete_mow_partial_update') { String db = context.config.getDbNameByFile(context.file) sql "select 1;" // to create database diff --git a/regression-test/suites/nereids_p0/explain/explain_dml.groovy b/regression-test/suites/nereids_p0/explain/explain_dml.groovy new file mode 100644 index 00000000000..a007185ce63 --- /dev/null +++ b/regression-test/suites/nereids_p0/explain/explain_dml.groovy @@ -0,0 +1,124 @@ +// 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("explain_dml") { + String db = context.config.getDbNameByFile(context.file) + sql "use ${db}" + multi_sql """ + set enable_nereids_planner=true; + set enable_fallback_to_original_planner=false; + drop table if exists epldel1; + CREATE TABLE epldel1 + (id INT, c1 BIGINT, c2 STRING, c3 DOUBLE, c4 DATE) + UNIQUE KEY (id) + DISTRIBUTED BY HASH (id) + PROPERTIES('replication_num'='1', "function_column.sequence_col" = "c4"); + + drop table if exists epldel2; + CREATE TABLE epldel2 + (id INT, c1 BIGINT, c2 STRING, c3 DOUBLE, c4 DATE) + DISTRIBUTED BY HASH (id) + PROPERTIES('replication_num'='1'); + + drop table if exists epldel3; + CREATE TABLE epldel3 + (id INT) + DISTRIBUTED BY HASH (id) + PROPERTIES('replication_num'='1'); + + INSERT INTO epldel1 VALUES + (1, 1, '1', 1.0, '2000-01-01'), + (2, 2, '2', 2.0, '2000-01-02'), + (3, 3, '3', 3.0, '2000-01-03'); + + INSERT INTO epldel2 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'); + + INSERT INTO epldel3 VALUES + (1), + (4), + (5); + + drop table if exists aggtbl; + CREATE TABLE `aggtbl` ( + `k1` int(11) NULL COMMENT "", + `v1` int(11) SUM DEFAULT "0", + `v2` int(11) SUM DEFAULT "0" + ) + aggregate key (k1) + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES ('replication_num' = '1'); + insert into aggtbl values (1, 1, 1); + + drop table if exists duptbl; + CREATE TABLE `duptbl` ( + `k1` int(11) NULL COMMENT "", + `v1` int(11) SUM DEFAULT "0", + `v2` int(11) SUM DEFAULT "0" + ) + aggregate key (k1) + DISTRIBUTED BY HASH(`k1`) BUCKETS 10 + PROPERTIES ('replication_num' = '1'); + insert into duptbl values (1,1,1); + """ + + explain { + sql "delete from epldel1 where id=0;" + contains "PLAN FRAGMENT 0" + } + + explain { + sql """ + DELETE FROM epldel1 + USING epldel2 INNER JOIN epldel3 ON epldel2.id = epldel3.id + WHERE epldel1.id = epldel2.id; + """ + contains "PLAN FRAGMENT 0" + } + + test { + sql "explain delete from aggtbl where v1=6;" + exception "delete command on aggregate/duplicate table is not explainable" + } + + test { + sql """ + explain DELETE FROM aggtbl + USING epldel2 INNER JOIN epldel3 ON epldel2.id = epldel3.id + WHERE aggtbl.k1 = epldel2.id;""" + exception "delete command on with using clause only supports unique key model" + } + + test { + sql "delete from aggtbl where v1=6;" + exception "delete predicate on value column only supports Unique table with merge-on-write enabled and Duplicate table, but Table[aggtbl] is an Aggregate table." + } + + test { + sql "update aggtbl set v1=1 where k1=1;" + exception "Only unique table could be updated." + } + + test { + sql "update duptbl set v1=1 where k1=1;" + exception "Only unique table could be updated." + } +} --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org