This is an automated email from the ASF dual-hosted git repository. morrysnow 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 5ae14549d1 [Feature](Nereids) support delete using syntax to delete data from unique key table (#20452) 5ae14549d1 is described below commit 5ae14549d1060be1a62af9fc1d6270b880617e3b Author: mch_ucchi <41606806+sohardforan...@users.noreply.github.com> AuthorDate: Sun Jun 18 16:22:21 2023 +0800 [Feature](Nereids) support delete using syntax to delete data from unique key table (#20452) --- .../antlr4/org/apache/doris/nereids/DorisParser.g4 | 4 + .../nereids/analyzer/UnboundOlapTableSink.java | 11 +- .../doris/nereids/parser/LogicalPlanBuilder.java | 27 ++++- .../apache/doris/nereids/trees/plans/PlanType.java | 3 +- .../trees/plans/commands/DeleteCommand.java | 131 +++++++++++++++++++++ .../trees/plans/logical/LogicalOlapTableSink.java | 9 +- .../plans/physical/PhysicalOlapTableSink.java | 9 +- .../trees/plans/visitor/CommandVisitor.java | 5 + .../java/org/apache/doris/nereids/util/Utils.java | 4 + .../nereids/trees/plans/DeleteCommandTest.java | 104 ++++++++++++++++ .../data/nereids_p0/delete/delete_using.out | 15 +++ .../suites/nereids_p0/delete/delete_using.groovy | 36 ++++++ .../suites/nereids_p0/delete/load.groovy | 84 +++++++++++++ 13 files changed, 418 insertions(+), 24 deletions(-) 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 b55b56c033..9d29436d6d 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 @@ -49,6 +49,10 @@ statement SET updateAssignmentSeq fromClause? whereClause #update + | explain? DELETE FROM tableName=multipartIdentifier tableAlias + (PARTITION partition=identifierList)? + (USING relation (COMMA relation)*) + whereClause #delete ; // -----------------Command accessories----------------- diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java index 872e35c7d9..cfebd0e5d3 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/analyzer/UnboundOlapTableSink.java @@ -26,6 +26,7 @@ import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.logical.LogicalUnary; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -53,9 +54,9 @@ public class UnboundOlapTableSink<CHILD_TYPE extends Plan> extends LogicalUnary< Optional<LogicalProperties> logicalProperties, CHILD_TYPE child) { super(PlanType.LOGICAL_UNBOUND_OLAP_TABLE_SINK, groupExpression, logicalProperties, child); this.nameParts = ImmutableList.copyOf(Objects.requireNonNull(nameParts, "nameParts cannot be null")); - this.colNames = copyIfNotNull(colNames); - this.hints = copyIfNotNull(hints); - this.partitions = copyIfNotNull(partitions); + this.colNames = Utils.copyIfNotNull(colNames); + this.hints = Utils.copyIfNotNull(hints); + this.partitions = Utils.copyIfNotNull(partitions); } public List<String> getColNames() { @@ -70,10 +71,6 @@ public class UnboundOlapTableSink<CHILD_TYPE extends Plan> extends LogicalUnary< return partitions; } - private <T> List<T> copyIfNotNull(List<T> list) { - return list == null ? null : ImmutableList.copyOf(list); - } - @Override public Plan withChildren(List<Plan> children) { Preconditions.checkArgument(children.size() == 1, "UnboundOlapTableSink only accepts one child"); 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 8ebd41af72..8492390979 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 @@ -42,6 +42,7 @@ import org.apache.doris.nereids.DorisParser.CteContext; import org.apache.doris.nereids.DorisParser.Date_addContext; import org.apache.doris.nereids.DorisParser.Date_subContext; import org.apache.doris.nereids.DorisParser.DecimalLiteralContext; +import org.apache.doris.nereids.DorisParser.DeleteContext; import org.apache.doris.nereids.DorisParser.DereferenceContext; import org.apache.doris.nereids.DorisParser.ExistContext; import org.apache.doris.nereids.DorisParser.ExplainContext; @@ -213,6 +214,7 @@ import org.apache.doris.nereids.trees.plans.algebra.Aggregate; import org.apache.doris.nereids.trees.plans.algebra.SetOperation.Qualifier; 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.DeleteCommand; 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; @@ -328,7 +330,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { RelationUtil.newRelationId(), visitMultipartIdentifier(ctx.tableName))); query = withTableAlias(query, ctx.tableAlias()); if (ctx.fromClause() != null) { - query = withRelations(query, ctx.fromClause()); + query = withRelations(query, ctx.fromClause().relation()); } query = withFilter(query, Optional.of(ctx.whereClause())); String tableAlias = null; @@ -339,6 +341,23 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { visitUpdateAssignmentSeq(ctx.updateAssignmentSeq()), query), ctx.explain()); } + @Override + public LogicalPlan visitDelete(DeleteContext ctx) { + List<String> tableName = visitMultipartIdentifier(ctx.tableName); + List<String> partitions = ctx.partition == null ? null : visitIdentifierList(ctx.partition); + LogicalPlan query = withTableAlias(withCheckPolicy( + new UnboundRelation(RelationUtil.newRelationId(), tableName)), ctx.tableAlias()); + if (ctx.USING() != null) { + query = withRelations(query, ctx.relation()); + } + query = withFilter(query, Optional.of(ctx.whereClause())); + String tableAlias = null; + if (ctx.tableAlias().strictIdentifier() != null) { + tableAlias = ctx.tableAlias().getText(); + } + return withExplain(new DeleteCommand(tableName, tableAlias, partitions, query), ctx.explain()); + } + /** * Visit multi-statements. */ @@ -1293,7 +1312,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { @Override public LogicalPlan visitFromClause(FromClauseContext ctx) { - return ParserUtils.withOrigin(ctx, () -> withRelations(null, ctx)); + return ParserUtils.withOrigin(ctx, () -> withRelations(null, ctx.relation())); } /* ******************************************************************************************** @@ -1646,9 +1665,9 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { }); } - private LogicalPlan withRelations(LogicalPlan inputPlan, FromClauseContext ctx) { + private LogicalPlan withRelations(LogicalPlan inputPlan, List<RelationContext> relations) { LogicalPlan left = inputPlan; - for (RelationContext relation : ctx.relation()) { + for (RelationContext relation : relations) { // build left deep join tree LogicalPlan right = visitRelation(relation); left = (left == null) ? right : diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java index b5e6219d7f..4671d95f60 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/PlanType.java @@ -106,5 +106,6 @@ public enum PlanType { EXPLAIN_COMMAND, CREATE_POLICY_COMMAND, INSERT_INTO_TABLE_COMMAND, - UPDATE_COMMAND + UPDATE_COMMAND, + DELETE_COMMAND } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java new file mode 100644 index 0000000000..48408db1a8 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/DeleteCommand.java @@ -0,0 +1,131 @@ +// 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.TableIf; +import org.apache.doris.nereids.analyzer.UnboundOlapTableSink; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Alias; +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.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.nereids.util.Utils; +import org.apache.doris.qe.ConnectContext; +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.Objects; + +/** + * delete from unique key table. + */ +public class DeleteCommand extends Command implements ForwardWithSync, Explainable { + private final List<String> nameParts; + private final String tableAlias; + private final List<String> partitions; + private LogicalPlan logicalQuery; + private OlapTable targetTable; + + /** + * constructor + */ + public DeleteCommand(List<String> nameParts, String tableAlias, List<String> partitions, LogicalPlan logicalQuery) { + super(PlanType.DELETE_COMMAND); + this.nameParts = ImmutableList.copyOf(Objects.requireNonNull(nameParts, + "nameParts in DeleteCommand cannot be null")); + this.tableAlias = tableAlias; + this.partitions = Utils.copyIfNotNull(partitions); + this.logicalQuery = logicalQuery; + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + new InsertIntoTableCommand(completeQueryPlan(ctx, logicalQuery), null).run(ctx, executor); + } + + private void checkTable(ConnectContext ctx) { + List<String> qualifieredTableName = RelationUtil.getQualifierName(ctx, nameParts); + TableIf table = RelationUtil.getTable(qualifieredTableName, ctx.getEnv()); + if (!(table instanceof OlapTable)) { + throw new AnalysisException("table must be olapTable in delete command"); + } + targetTable = ((OlapTable) table); + if (targetTable.getKeysType() != KeysType.UNIQUE_KEYS) { + throw new AnalysisException("Nereids only support delete command on unique key table now"); + } + } + + /** + * public for test + */ + public LogicalPlan completeQueryPlan(ConnectContext ctx, LogicalPlan logicalQuery) { + checkTable(ctx); + + // 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 Alias(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); + + // make UnboundTableSink + return new UnboundOlapTableSink<>(nameParts, cols, null, partitions, logicalQuery); + } + + public LogicalPlan getLogicalQuery() { + return logicalQuery; + } + + @Override + public Plan getExplainPlan(ConnectContext ctx) { + return completeQueryPlan(ctx, logicalQuery); + } + + @Override + public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { + return visitor.visitDeleteCommand(this, context); + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java index fb7f2e11a7..254d503a80 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/logical/LogicalOlapTableSink.java @@ -27,6 +27,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; import com.google.common.base.Preconditions; import com.google.common.collect.ImmutableList; @@ -56,8 +57,8 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalUnary< super(PlanType.LOGICAL_OLAP_TABLE_SINK, groupExpression, logicalProperties, child); this.database = Objects.requireNonNull(database, "database != null in LogicalOlapTableSink"); this.targetTable = Objects.requireNonNull(targetTable, "targetTable != null in LogicalOlapTableSink"); - this.cols = copyIfNotNull(cols); - this.partitionIds = copyIfNotNull(partitionIds); + this.cols = Utils.copyIfNotNull(cols); + this.partitionIds = Utils.copyIfNotNull(partitionIds); } @Override @@ -83,10 +84,6 @@ public class LogicalOlapTableSink<CHILD_TYPE extends Plan> extends LogicalUnary< return partitionIds; } - private <T> List<T> copyIfNotNull(List<T> list) { - return list == null ? null : ImmutableList.copyOf(list); - } - @Override public boolean equals(Object o) { if (this == o) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java index 1bcb30fabb..6b9c39fff1 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/physical/PhysicalOlapTableSink.java @@ -30,6 +30,7 @@ import org.apache.doris.nereids.trees.expressions.Slot; import org.apache.doris.nereids.trees.plans.Plan; import org.apache.doris.nereids.trees.plans.PlanType; import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.util.Utils; import org.apache.doris.statistics.Statistics; import com.google.common.base.Preconditions; @@ -82,8 +83,8 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalUnar statistics, child); this.database = Objects.requireNonNull(database, "database != null in PhysicalOlapTableSink"); this.targetTable = Objects.requireNonNull(targetTable, "targetTable != null in PhysicalOlapTableSink"); - this.cols = copyIfNotNull(cols); - this.partitionIds = copyIfNotNull(partitionIds); + this.cols = Utils.copyIfNotNull(cols); + this.partitionIds = Utils.copyIfNotNull(partitionIds); this.singleReplicaLoad = singleReplicaLoad; } @@ -107,10 +108,6 @@ public class PhysicalOlapTableSink<CHILD_TYPE extends Plan> extends PhysicalUnar return singleReplicaLoad; } - private <T> List<T> copyIfNotNull(List<T> list) { - return list == null ? null : ImmutableList.copyOf(list); - } - @Override public Plan withChildren(List<Plan> children) { Preconditions.checkArgument(children.size() == 1, "PhysicalOlapTableSink only accepts one child"); 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 index 5adab2cc96..264598d928 100644 --- 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 @@ -19,6 +19,7 @@ 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.DeleteCommand; 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; @@ -43,4 +44,8 @@ public interface CommandVisitor<R, C> { default R visitUpdateCommand(UpdateCommand updateCommand, C context) { return visitCommand(updateCommand, context); } + + default R visitDeleteCommand(DeleteCommand deleteCommand, C context) { + return visitCommand(deleteCommand, context); + } } diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java index 04ab5eb395..83288e2456 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/util/Utils.java @@ -251,4 +251,8 @@ public class Utils { ) ).collect(ImmutableList.toImmutableList()); } + + public static <T> List<T> copyIfNotNull(List<T> list) { + return list == null ? null : ImmutableList.copyOf(list); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java new file mode 100644 index 0000000000..12a7ba48ce --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/DeleteCommandTest.java @@ -0,0 +1,104 @@ +// 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.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.trees.plans.commands.DeleteCommand; +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 DeleteCommandTest extends TestWithFeService implements PlanPatternMatchSupported { + @Override + protected 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 testFromClauseUpdate() throws AnalysisException { + String sql = "delete from t1 a using src join t2 on src.k1 = t2.k1 where t2.k1 = a.k1"; + LogicalPlan parsed = new NereidsParser().parseSingle(sql); + Assertions.assertTrue(parsed instanceof DeleteCommand); + DeleteCommand command = ((DeleteCommand) 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/delete/delete_using.out b/regression-test/data/nereids_p0/delete/delete_using.out new file mode 100644 index 0000000000..ac92b46e48 --- /dev/null +++ b/regression-test/data/nereids_p0/delete/delete_using.out @@ -0,0 +1,15 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !sql -- +1 \N 2 1 1.0 \N +1 10 1 1 1.0 2000-01-01 +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + +-- !sql -- +2 \N 4 2 2.0 \N +2 20 2 2 2.0 2000-01-02 +3 \N 6 3 3.0 \N +3 30 3 3 3.0 2000-01-03 + diff --git a/regression-test/suites/nereids_p0/delete/delete_using.groovy b/regression-test/suites/nereids_p0/delete/delete_using.groovy new file mode 100644 index 0000000000..ff1b1e0336 --- /dev/null +++ b/regression-test/suites/nereids_p0/delete/delete_using.groovy @@ -0,0 +1,36 @@ +// 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('nereids_delete_using') { + sql 'set enable_nereids_planner=true' + sql 'set enable_fallback_to_original_planner=false' + sql 'set enable_nereids_dml=true' + + sql 'insert into t1(id, c1, c2, c3) select id, c1 * 2, c2, c3 from t1' + sql 'insert into t2(id, c1, c2, c3) select id, c1, c2 * 2, c3 from t2' + sql 'insert into t2(c1, c3) select c1 + 1, c3 + 1 from (select id, c1, c3 from t1 order by id, c1 limit 10) t1, t3' + + qt_sql 'select * from t1 order by id, id1' + + sql ''' + delete from t1 + using t2 join t3 on t2.id = t3.id + where t1.id = t2.id; + ''' + + qt_sql 'select * from t1 order by id, id1' +} diff --git a/regression-test/suites/nereids_p0/delete/load.groovy b/regression-test/suites/nereids_p0/delete/load.groovy new file mode 100644 index 0000000000..6415361b09 --- /dev/null +++ b/regression-test/suites/nereids_p0/delete/load.groovy @@ -0,0 +1,84 @@ +// 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 'drop table if exists t1' + 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 'drop table if exists t2' + 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 'drop table if exists t3' + 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); + ''' +} --------------------------------------------------------------------- To unsubscribe, e-mail: commits-unsubscr...@doris.apache.org For additional commands, e-mail: commits-h...@doris.apache.org