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 <[email protected]>
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: [email protected]
For additional commands, e-mail: [email protected]