This is an automated email from the ASF dual-hosted git repository. morrysnow pushed a commit to branch pick_4.0_merge_into in repository https://gitbox.apache.org/repos/asf/doris.git
commit f25d0fc98711c6a06972ef73a81216041a8cc988 Author: morrySnow <[email protected]> AuthorDate: Fri Oct 24 11:46:07 2025 +0800 [feat](load) support merge into (#57044) support merge into syntax [cte] MERGE INTO <target_table> USING <source> ON <join_expr> { matchedClause | notMatchedClause } [ ... ] matchedClause ::= WHEN MATCHED [ AND <case_predicate> ] THEN { UPDATE SET <col_name> = <expr> [ , <col_name> = <expr> ... ] | DELETE } [ ... ] notMatchedClause ::= WHEN NOT MATCHED [ AND <case_predicate> ] THEN INSERT [ ( <col_name> [ , ... ] ) ] VALUES ( <expr> [ , ... ] ) --- fe/.idea/vcs.xml | 24 +- .../antlr4/org/apache/doris/nereids/DorisLexer.g4 | 1 + .../antlr4/org/apache/doris/nereids/DorisParser.g4 | 15 + .../java/org/apache/doris/analysis/StmtType.java | 1 + .../main/java/org/apache/doris/catalog/Column.java | 4 + .../java/org/apache/doris/catalog/OlapTable.java | 4 + .../doris/nereids/parser/LogicalPlanBuilder.java | 51 +- .../doris/nereids/rules/analysis/BindSink.java | 4 +- .../apache/doris/nereids/trees/plans/PlanType.java | 1 + .../trees/plans/commands/UpdateCommand.java | 22 +- .../plans/commands/merge/MergeIntoCommand.java | 530 ++++++++++++ .../plans/commands/merge/MergeMatchedClause.java | 55 ++ .../commands/merge/MergeNotMatchedClause.java | 55 ++ .../trees/plans/visitor/CommandVisitor.java | 5 + .../doris/nereids/parser/NereidsParserTest.java | 94 ++ .../plans/commands/merge/MergeIntoCommandTest.java | 959 +++++++++++++++++++++ .../data/load_p0/merge_into/test_merge_into.out | 21 + .../load_p0/merge_into/test_merge_into.groovy | 254 ++++++ 18 files changed, 2072 insertions(+), 28 deletions(-) diff --git a/fe/.idea/vcs.xml b/fe/.idea/vcs.xml index 7b2cdb1cbbd..e5158c553f8 100644 --- a/fe/.idea/vcs.xml +++ b/fe/.idea/vcs.xml @@ -1,20 +1,4 @@ <?xml version="1.0" encoding="UTF-8"?> -<!-- - 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. - --> <project version="4"> <component name="IssueNavigationConfiguration"> <option name="links"> @@ -27,6 +11,10 @@ </option> </component> <component name="VcsDirectoryMappings"> - <mapping directory="$PROJECT_DIR$/.." vcs="Git" /> + <mapping directory="$PROJECT_DIR$/.." vcs="Git" /> + <mapping directory="$PROJECT_DIR$/../contrib/apache-orc" vcs="Git" /> + <mapping directory="$PROJECT_DIR$/../contrib/clucene" vcs="Git" /> + <mapping directory="$PROJECT_DIR$/../contrib/faiss" vcs="Git" /> + <mapping directory="$PROJECT_DIR$/../contrib/openblas" vcs="Git" /> </component> -</project> +</project> \ No newline at end of file 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 521ef91040b..ca33b65e552 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 @@ -340,6 +340,7 @@ LOW_PRIORITY: 'LOW_PRIORITY'; MANUAL: 'MANUAL'; MAP: 'MAP'; MATCH: 'MATCH'; +MATCHED: 'MATCHED'; MATCH_ALL: 'MATCH_ALL'; MATCH_ANY: 'MATCH_ANY'; MATCH_NAME: 'MATCH_NAME'; 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 38fe0e9247b..2113b806c04 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 @@ -145,6 +145,10 @@ supportedDmlStatement partitionSpec? tableAlias (USING relations)? whereClause? #delete + | explain? cte? MERGE INTO targetTable=multipartIdentifier + (AS? identifier)? USING srcRelation=relationPrimary + ON expression + (mergeMatchedClause | mergeNotMatchedClause)+ #mergeInto | LOAD LABEL lableName=multipartIdentifier LEFT_PAREN dataDescs+=dataDesc (COMMA dataDescs+=dataDesc)* RIGHT_PAREN (withRemoteStorageSystem)? @@ -164,6 +168,16 @@ supportedDmlStatement | TRUNCATE TABLE multipartIdentifier specifiedPartition? FORCE? #truncateTable ; +mergeMatchedClause + : WHEN MATCHED (AND casePredicate=expression)? THEN + (UPDATE SET updateAssignmentSeq | DELETE) + ; + +mergeNotMatchedClause + : WHEN NOT MATCHED (AND casePredicate=expression)? THEN + INSERT cols=identifierList? VALUES rowConstructor + ; + supportedCreateStatement : CREATE (EXTERNAL | TEMPORARY)? TABLE (IF NOT EXISTS)? name=multipartIdentifier ((ctasCols=identifierList)? | (LEFT_PAREN columnDefs (COMMA indexDefs)? COMMA? RIGHT_PAREN)) @@ -2057,6 +2071,7 @@ nonReserved | LOGICAL | MANUAL | MAP + | MATCHED | MATCH_ALL | MATCH_ANY | MATCH_PHRASE diff --git a/fe/fe-core/src/main/java/org/apache/doris/analysis/StmtType.java b/fe/fe-core/src/main/java/org/apache/doris/analysis/StmtType.java index b929f198c77..af9537cf8c6 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/analysis/StmtType.java +++ b/fe/fe-core/src/main/java/org/apache/doris/analysis/StmtType.java @@ -38,6 +38,7 @@ public enum StmtType { INSTALL, KILL, LOAD, + MERGE_INTO, OTHER, OPTIMIZE, PAUSE, diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java index ac2d2fd53cd..6bd014aec59 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/Column.java @@ -1288,6 +1288,10 @@ public class Column implements GsonPostProcessable { return colName; } + public boolean isGeneratedColumn() { + return generatedColumnInfo != null; + } + public GeneratedColumnInfo getGeneratedColumnInfo() { return generatedColumnInfo; } diff --git a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java index bda8f0f8122..f3fb4914ff5 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java +++ b/fe/fe-core/src/main/java/org/apache/doris/catalog/OlapTable.java @@ -1691,6 +1691,10 @@ public class OlapTable extends Table implements MTMVRelatedTableIf, GsonPostProc return getBaseSchema().stream().anyMatch(column -> !column.isVisible()); } + public boolean hasGeneratedColumn() { + return getBaseSchema().stream().anyMatch(Column::isGeneratedColumn); + } + public Type getSequenceType() { if (getSequenceCol() == null) { return null; 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 c43718126c6..2c7ac74c0ba 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 @@ -256,6 +256,9 @@ import org.apache.doris.nereids.DorisParser.LockTablesContext; import org.apache.doris.nereids.DorisParser.LogicalBinaryContext; import org.apache.doris.nereids.DorisParser.LogicalNotContext; import org.apache.doris.nereids.DorisParser.MapLiteralContext; +import org.apache.doris.nereids.DorisParser.MergeIntoContext; +import org.apache.doris.nereids.DorisParser.MergeMatchedClauseContext; +import org.apache.doris.nereids.DorisParser.MergeNotMatchedClauseContext; import org.apache.doris.nereids.DorisParser.ModifyColumnClauseContext; import org.apache.doris.nereids.DorisParser.ModifyColumnCommentClauseContext; import org.apache.doris.nereids.DorisParser.ModifyDistributionClauseContext; @@ -1000,6 +1003,9 @@ import org.apache.doris.nereids.trees.plans.commands.load.PauseRoutineLoadComman import org.apache.doris.nereids.trees.plans.commands.load.ResumeRoutineLoadCommand; import org.apache.doris.nereids.trees.plans.commands.load.ShowCreateRoutineLoadCommand; import org.apache.doris.nereids.trees.plans.commands.load.StopRoutineLoadCommand; +import org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand; +import org.apache.doris.nereids.trees.plans.commands.merge.MergeMatchedClause; +import org.apache.doris.nereids.trees.plans.commands.merge.MergeNotMatchedClause; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshCatalogCommand; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshDictionaryCommand; @@ -1372,6 +1378,49 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { return withExplain(command, ctx.explain()); } + @Override + public Object visitMergeInto(MergeIntoContext ctx) { + return ParserUtils.withOrigin(ctx, () -> { + List<String> targetNameParts = visitMultipartIdentifier(ctx.targetTable); + Optional<String> targetAlias = Optional.ofNullable( + ctx.identifier() != null ? ctx.identifier().getText() : null); + LogicalPlan source = plan(ctx.relationPrimary()); + Expression onClause = typedVisit(ctx.expression()); + List<MergeMatchedClause> matchedClauses = visit(ctx.mergeMatchedClause(), MergeMatchedClause.class); + List<MergeNotMatchedClause> notMatchedClauses = visit(ctx.mergeNotMatchedClause(), + MergeNotMatchedClause.class); + Optional<LogicalPlan> cte = Optional.empty(); + if (ctx.cte() != null) { + cte = Optional.ofNullable(withCte(source, ctx.cte())); + } + return withExplain(new MergeIntoCommand(targetNameParts, targetAlias, cte, + source, onClause, matchedClauses, notMatchedClauses), ctx.explain()); + }); + } + + @Override + public MergeMatchedClause visitMergeMatchedClause(MergeMatchedClauseContext ctx) { + return ParserUtils.withOrigin(ctx, () -> { + Optional<Expression> casePredicate = Optional.ofNullable( + ctx.casePredicate != null ? typedVisit(ctx.casePredicate) : null); + boolean isDelete = ctx.DELETE() != null; + List<EqualTo> updateAssignments = isDelete ? ImmutableList.of() : + visitUpdateAssignmentSeq(ctx.updateAssignmentSeq()); + return new MergeMatchedClause(casePredicate, updateAssignments, isDelete); + }); + } + + @Override + public MergeNotMatchedClause visitMergeNotMatchedClause(MergeNotMatchedClauseContext ctx) { + return ParserUtils.withOrigin(ctx, () -> { + Optional<Expression> casePredicate = Optional.ofNullable( + ctx.casePredicate != null ? typedVisit(ctx.casePredicate) : null); + List<String> cols = ctx.cols != null ? visitIdentifierList(ctx.cols) : ImmutableList.of(); + List<NamedExpression> row = visitRowConstructor(ctx.rowConstructor()); + return new MergeNotMatchedClause(casePredicate, cols, row); + }); + } + /** * return a pair, first will be true if partitions is temp partition, select is a list to present partition list. */ @@ -2445,7 +2494,7 @@ public class LogicalPlanBuilder extends DorisParserBaseVisitor<Object> { } @Override - public LogicalPlan visitInlineTable(InlineTableContext ctx) { + public UnboundInlineTable visitInlineTable(InlineTableContext ctx) { List<RowConstructorContext> rowConstructorContexts = ctx.rowConstructor(); ImmutableList.Builder<List<NamedExpression>> rows = ImmutableList.builderWithExpectedSize(rowConstructorContexts.size()); diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java index 582b048ceb3..bb75f846d51 100644 --- a/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/rules/analysis/BindSink.java @@ -354,7 +354,7 @@ public class BindSink implements AnalysisRuleFactory { List<Column> shadowColumns = Lists.newArrayList(); // generate slots not mentioned in sql, mv slots and shaded slots. for (Column column : boundSink.getTargetTable().getFullSchema()) { - if (column.getGeneratedColumnInfo() != null) { + if (column.isGeneratedColumn()) { generatedColumns.add(column); continue; } else if (column.isMaterializedViewColumn()) { @@ -814,7 +814,7 @@ public class BindSink implements AnalysisRuleFactory { ++extraColumnsNum; processedColsName.add(col.getName()); } - } else if (col.getGeneratedColumnInfo() != null) { + } else if (col.isGeneratedColumn()) { ++extraColumnsNum; processedColsName.add(col.getName()); } 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 f50bd047f3a..f2e88c1df7d 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 @@ -147,6 +147,7 @@ public enum PlanType { DROP_DICTIONARY_COMMAND, CREATE_SQL_BLOCK_RULE_COMMAND, DELETE_COMMAND, + MERGE_INTO_COMMAND, EXPLAIN_COMMAND, EXPLAIN_DICTIONARY_COMMAND, EXPORT_COMMAND, 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 index c439af0625b..133122ad91c 100644 --- 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 @@ -111,10 +111,10 @@ public class UpdateCommand extends Command implements ForwardWithSync, Explainab Map<String, Expression> colNameToExpression = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); Map<String, Expression> partialUpdateColNameToExpression = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); for (EqualTo equalTo : assignments) { - List<String> nameParts = ((UnboundSlot) equalTo.left()).getNameParts(); - checkAssignmentColumn(ctx, nameParts); - colNameToExpression.put(nameParts.get(nameParts.size() - 1), equalTo.right()); - partialUpdateColNameToExpression.put(nameParts.get(nameParts.size() - 1), equalTo.right()); + List<String> colNameParts = ((UnboundSlot) equalTo.left()).getNameParts(); + checkAssignmentColumn(ctx, colNameParts, this.nameParts, this.tableAlias); + colNameToExpression.put(colNameParts.get(colNameParts.size() - 1), equalTo.right()); + partialUpdateColNameToExpression.put(colNameParts.get(colNameParts.size() - 1), equalTo.right()); } // check if any key in update clause if (targetTable.getFullSchema().stream().filter(Column::isKey) @@ -198,7 +198,15 @@ public class UpdateCommand extends Command implements ForwardWithSync, Explainab DMLCommandType.UPDATE, logicalQuery); } - private void checkAssignmentColumn(ConnectContext ctx, List<String> columnNameParts) { + /** + * check assignment column valid or not. + * @param ctx connect context + * @param columnNameParts qualified column name + * @param tableNameParts qualified target table name + * @param tableAlias target table alias + */ + public static void checkAssignmentColumn(ConnectContext ctx, List<String> columnNameParts, + List<String> tableNameParts, String tableAlias) { if (columnNameParts.size() <= 1) { return; } @@ -212,10 +220,10 @@ public class UpdateCommand extends Command implements ForwardWithSync, Explainab } else { throw new AnalysisException("column in assignment list is invalid, " + String.join(".", columnNameParts)); } - if (dbName != null && this.tableAlias != null) { + if (dbName != null && tableAlias != null) { throw new AnalysisException("column in assignment list is invalid, " + String.join(".", columnNameParts)); } - List<String> tableQualifier = RelationUtil.getQualifierName(ctx, nameParts); + List<String> tableQualifier = RelationUtil.getQualifierName(ctx, tableNameParts); if (!ExpressionAnalyzer.sameTableName(tableAlias == null ? tableQualifier.get(2) : tableAlias, tableName) || (dbName != null && !ExpressionAnalyzer.compareDbNameIgnoreClusterName(tableQualifier.get(1), dbName))) { diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeIntoCommand.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeIntoCommand.java new file mode 100644 index 00000000000..38af17d8508 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeIntoCommand.java @@ -0,0 +1,530 @@ +// 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.merge; + +import org.apache.doris.analysis.ColumnDef.DefaultValue; +import org.apache.doris.analysis.StmtType; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.TableIf; +import org.apache.doris.catalog.Type; +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.UnboundStar; +import org.apache.doris.nereids.analyzer.UnboundTableSinkCreator; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.parser.LogicalPlanBuilderAssistant; +import org.apache.doris.nereids.parser.NereidsParser; +import org.apache.doris.nereids.rules.exploration.join.JoinReorderContext; +import org.apache.doris.nereids.trees.expressions.Alias; +import org.apache.doris.nereids.trees.expressions.Cast; +import org.apache.doris.nereids.trees.expressions.DefaultValueSlot; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +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.StatementScopeIdGenerator; +import org.apache.doris.nereids.trees.expressions.functions.scalar.If; +import org.apache.doris.nereids.trees.expressions.functions.scalar.Now; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; +import org.apache.doris.nereids.trees.plans.Explainable; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.trees.plans.Plan; +import org.apache.doris.nereids.trees.plans.PlanType; +import org.apache.doris.nereids.trees.plans.commands.Command; +import org.apache.doris.nereids.trees.plans.commands.ForwardWithSync; +import org.apache.doris.nereids.trees.plans.commands.UpdateCommand; +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.LogicalFilter; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +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.logical.LogicalSubQueryAlias; +import org.apache.doris.nereids.trees.plans.visitor.PlanVisitor; +import org.apache.doris.nereids.types.DataType; +import org.apache.doris.nereids.types.IntegerType; +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 org.apache.doris.thrift.TPartialUpdateNewRowPolicy; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableSet; +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 java.util.Optional; + +/** + * merge into table + */ +public class MergeIntoCommand extends Command implements ForwardWithSync, Explainable { + private static final String BRANCH_LABEL = "__DORIS_MERGE_INTO_BRANCH_LABEL__"; + + private final List<String> targetNameParts; + private final Optional<String> targetAlias; + private final List<String> targetNameInPlan; + private final Optional<LogicalPlan> cte; + private final LogicalPlan source; + private final Expression onClause; + private final List<MergeMatchedClause> matchedClauses; + private final List<MergeNotMatchedClause> notMatchedClauses; + + /** + * constructor. + */ + public MergeIntoCommand(List<String> targetNameParts, Optional<String> targetAlias, + Optional<LogicalPlan> cte, LogicalPlan source, + Expression onClause, List<MergeMatchedClause> matchedClauses, + List<MergeNotMatchedClause> notMatchedClauses) { + super(PlanType.MERGE_INTO_COMMAND); + this.targetNameParts = Utils.fastToImmutableList( + Objects.requireNonNull(targetNameParts, "targetNameParts should not be null")); + this.targetAlias = Objects.requireNonNull(targetAlias, "targetAlias should not be null"); + if (targetAlias.isPresent()) { + this.targetNameInPlan = ImmutableList.of(targetAlias.get()); + } else { + this.targetNameInPlan = ImmutableList.copyOf(targetNameParts); + } + this.cte = Objects.requireNonNull(cte, "cte should not be null"); + this.source = Objects.requireNonNull(source, "source should not be null"); + this.onClause = Objects.requireNonNull(onClause, "onClause should not be null"); + this.matchedClauses = Utils.fastToImmutableList( + Objects.requireNonNull(matchedClauses, "matchedClauses should not be null")); + this.notMatchedClauses = Utils.fastToImmutableList( + Objects.requireNonNull(notMatchedClauses, "notMatchedClauses should not be null")); + } + + @Override + public void run(ConnectContext ctx, StmtExecutor executor) throws Exception { + new InsertIntoTableCommand(completeQueryPlan(ctx), Optional.empty(), Optional.empty(), + Optional.empty(), true, Optional.empty()).run(ctx, executor); + } + + @Override + public <R, C> R accept(PlanVisitor<R, C> visitor, C context) { + return visitor.visitMergeIntoCommand(this, context); + } + + @Override + public Plan getExplainPlan(ConnectContext ctx) { + return completeQueryPlan(ctx); + } + + private OlapTable getTargetTable(ConnectContext ctx) { + List<String> qualifiedTableName = RelationUtil.getQualifierName(ctx, targetNameParts); + TableIf table = RelationUtil.getTable(qualifiedTableName, ctx.getEnv(), Optional.empty()); + if (!(table instanceof OlapTable) || !((OlapTable) table).getEnableUniqueKeyMergeOnWrite()) { + throw new AnalysisException("merge into command only support MOW unique key olapTable"); + } + return ((OlapTable) table); + } + + @Override + public StmtType stmtType() { + return StmtType.MERGE_INTO; + } + + /** + * generate target right outer join source. + */ + private LogicalPlan generateBasePlan() { + LogicalPlan plan = LogicalPlanBuilderAssistant.withCheckPolicy( + new UnboundRelation( + StatementScopeIdGenerator.newRelationId(), + targetNameParts + ) + ); + if (targetAlias.isPresent()) { + plan = new LogicalSubQueryAlias<>(targetAlias.get(), plan); + } + return new LogicalJoin<>(JoinType.LEFT_OUTER_JOIN, + ImmutableList.of(), ImmutableList.of(onClause), + source, plan, JoinReorderContext.EMPTY); + } + + /** + * generate a branch number column to indicate this row matched witch branch + */ + private NamedExpression generateBranchLabel(NamedExpression deleteSign) { + Expression matchedLabel = new NullLiteral(IntegerType.INSTANCE); + for (int i = matchedClauses.size() - 1; i >= 0; i--) { + MergeMatchedClause clause = matchedClauses.get(i); + if (i != matchedClauses.size() - 1 && !clause.getCasePredicate().isPresent()) { + throw new AnalysisException("Only the last matched clause could without case predicate."); + } + Expression currentResult = new IntegerLiteral(i); + if (clause.getCasePredicate().isPresent()) { + matchedLabel = new If(clause.getCasePredicate().get(), currentResult, matchedLabel); + } else { + matchedLabel = currentResult; + } + } + Expression notMatchedLabel = new NullLiteral(IntegerType.INSTANCE); + for (int i = notMatchedClauses.size() - 1; i >= 0; i--) { + MergeNotMatchedClause clause = notMatchedClauses.get(i); + if (i != notMatchedClauses.size() - 1 && !clause.getCasePredicate().isPresent()) { + throw new AnalysisException("Only the last not matched clause could without case predicate."); + } + Expression currentResult = new IntegerLiteral(i + matchedClauses.size()); + if (clause.getCasePredicate().isPresent()) { + notMatchedLabel = new If(clause.getCasePredicate().get(), currentResult, notMatchedLabel); + } else { + notMatchedLabel = currentResult; + } + } + return new UnboundAlias(new If(new Not(new IsNull(deleteSign)), + matchedLabel, notMatchedLabel), BRANCH_LABEL); + } + + private List<Expression> generateDeleteProjection(List<Column> columns) { + ImmutableList.Builder<Expression> builder = ImmutableList.builder(); + for (Column column : columns) { + // delete + if (column.isDeleteSignColumn()) { + builder.add(new TinyIntLiteral(((byte) 1))); + } else if ((!column.isVisible() && !column.isSequenceColumn()) || column.isGeneratedColumn()) { + // skip this column + continue; + } else { + List<String> nameParts = Lists.newArrayList(targetNameInPlan); + nameParts.add(column.getName()); + builder.add(new UnboundSlot(nameParts)); + } + } + return builder.build(); + } + + private List<Expression> generateUpdateProjection(MergeMatchedClause clause, + List<Column> columns, OlapTable targetTable, ConnectContext ctx) { + ImmutableList.Builder<Expression> builder = ImmutableList.builder(); + Map<String, Expression> colNameToExpression = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + // update + for (EqualTo equalTo : clause.getAssignments()) { + List<String> nameParts = ((UnboundSlot) equalTo.left()).getNameParts(); + UpdateCommand.checkAssignmentColumn(ctx, nameParts, targetNameParts, targetAlias.orElse(null)); + if (colNameToExpression.put(nameParts.get(nameParts.size() - 1), equalTo.right()) != null) { + throw new AnalysisException("Duplicate column name in update: " + nameParts.get(nameParts.size() - 1)); + } + } + for (Column column : columns) { + DataType dataType = DataType.fromCatalogType(column.getType()); + if (colNameToExpression.containsKey(column.getName())) { + if (column.isKey()) { + throw new AnalysisException("Only value columns of unique table could be updated"); + } + if (column.isGeneratedColumn()) { + throw new AnalysisException("The value specified for generated column '" + + column.getName() + "' in table '" + targetTable.getName() + "' is not allowed."); + } + builder.add(new Cast(colNameToExpression.get(column.getName()), dataType)); + colNameToExpression.remove(column.getName()); + } else if (column.isGeneratedColumn() || (!column.isVisible() + && !column.isDeleteSignColumn() && !column.isSequenceColumn())) { + // skip these columns + continue; + } else if (column.hasOnUpdateDefaultValue()) { + builder.add(new Cast(new NereidsParser().parseExpression( + column.getOnUpdateDefaultValueSql()), dataType)); + } else { + List<String> nameParts = Lists.newArrayList(targetNameInPlan); + nameParts.add(column.getName()); + builder.add(new Cast(new UnboundSlot(nameParts), dataType)); + } + } + if (!colNameToExpression.isEmpty()) { + throw new AnalysisException("unknown column in assignment list: " + + String.join(", ", colNameToExpression.keySet())); + } + return builder.build(); + } + + private List<Expression> generateInsertWithoutColListProjection(MergeNotMatchedClause clause, + List<Column> columns, OlapTable targetTable, boolean hasSequenceCol, int seqColumnIndex, + Optional<Column> seqMappingColInTable, Optional<Type> seqColType) { + ImmutableList.Builder<Expression> builder = ImmutableList.builder(); + if (hasSequenceCol && seqColumnIndex < 0) { + if ((!seqMappingColInTable.isPresent() || seqMappingColInTable.get().getDefaultValue() == null + || !seqMappingColInTable.get().getDefaultValue() + .equalsIgnoreCase(DefaultValue.CURRENT_TIMESTAMP))) { + throw new AnalysisException("Table " + targetTable.getName() + + " has sequence column, need to specify the sequence column"); + } + } + Expression sqlColExpr = new Now(); + for (int i = 0; i < clause.getRow().size(); i++) { + DataType columnType = DataType.fromCatalogType(columns.get(i).getType()); + NamedExpression rowItem = clause.getRow().get(i); + Expression value; + if (rowItem instanceof Alias || rowItem instanceof UnboundAlias) { + value = rowItem.child(0); + } else { + value = rowItem; + } + if (columns.get(i).isGeneratedColumn()) { + if (!(value instanceof DefaultValueSlot)) { + throw new AnalysisException("The value specified for generated column '" + + columns.get(i).getName() + + "' in table '" + targetTable.getName() + "' is not allowed."); + } + continue; + } + value = new Cast(value, columnType); + if (i == seqColumnIndex) { + sqlColExpr = value; + } + builder.add(value); + } + // delete sign + builder.add(new TinyIntLiteral(((byte) 0))); + // sequence column + if (hasSequenceCol) { + builder.add(new Cast(sqlColExpr, seqColType.map(DataType::fromCatalogType).get())); + } + return builder.build(); + } + + private List<Expression> generateInsertWithColListProjection(MergeNotMatchedClause clause, + List<Column> columns, OlapTable targetTable, boolean hasSequenceCol, + String seqColumnName, Optional<Column> seqMappingColInTable, Optional<Type> seqColType) { + ImmutableList.Builder<Expression> builder = ImmutableList.builder(); + Map<String, Expression> colNameToExpression = Maps.newTreeMap(String.CASE_INSENSITIVE_ORDER); + for (int i = 0; i < clause.getColNames().size(); i++) { + String targetColumnName = clause.getColNames().get(i); + NamedExpression rowItem = clause.getRow().get(i); + if (rowItem instanceof Alias || rowItem instanceof UnboundAlias) { + colNameToExpression.put(targetColumnName, rowItem.child(0)); + } else { + colNameToExpression.put(targetColumnName, rowItem); + } + } + if (colNameToExpression.size() != clause.getColNames().size()) { + throw new AnalysisException("insert has duplicate column names"); + } + if (hasSequenceCol) { + if (seqColumnName == null || seqColumnName.isEmpty()) { + seqColumnName = Column.SEQUENCE_COL; + } + if (!colNameToExpression.containsKey(seqColumnName) + && (!seqMappingColInTable.isPresent() || seqMappingColInTable.get().getDefaultValue() == null + || !seqMappingColInTable.get().getDefaultValue() + .equalsIgnoreCase(DefaultValue.CURRENT_TIMESTAMP))) { + throw new AnalysisException("Table " + targetTable.getName() + + " has sequence column, need to specify the sequence column"); + } + } + for (Column column : columns) { + DataType type = DataType.fromCatalogType(column.getType()); + if (column.isGeneratedColumn()) { + if (colNameToExpression.containsKey(column.getName())) { + if (!(colNameToExpression.get(column.getName()) instanceof DefaultValueSlot)) { + throw new AnalysisException("The value specified for generated column '" + + column.getName() + "' in table '" + targetTable.getName() + "' is not allowed."); + } + colNameToExpression.remove(column.getName()); + } + continue; + } else if (!column.isVisible()) { + // skip these columns + continue; + } else if (colNameToExpression.containsKey(column.getName())) { + builder.add(new Cast(colNameToExpression.get(column.getName()), type)); + if (!column.getName().equalsIgnoreCase(seqColumnName)) { + colNameToExpression.remove(column.getName()); + } + } else { + if (!column.hasDefaultValue()) { + if (!column.isAllowNull() && !column.isAutoInc()) { + throw new AnalysisException("Column has no default value," + + " column=" + column.getName()); + } + builder.add(new NullLiteral(type)); + } else { + Expression defaultExpr; + try { + // it comes from the original planner, if default value expression is + // null, we use the literal string of the default value, or it may be + // default value function, like CURRENT_TIMESTAMP. + Expression unboundDefaultValue = new NereidsParser().parseExpression( + column.getDefaultValueSql()); + if (unboundDefaultValue instanceof UnboundAlias) { + unboundDefaultValue = ((UnboundAlias) unboundDefaultValue).child(); + } + defaultExpr = new Cast(unboundDefaultValue, type); + } catch (Exception e) { + throw new AnalysisException(e.getMessage(), e.getCause()); + } + builder.add(defaultExpr); + } + } + } + builder.add(colNameToExpression.getOrDefault(Column.DELETE_SIGN, new TinyIntLiteral(((byte) 0)))); + colNameToExpression.remove(Column.DELETE_SIGN); + if (hasSequenceCol) { + Expression forSeqCol; + if (colNameToExpression.containsKey(Column.SEQUENCE_COL)) { + forSeqCol = colNameToExpression.get(Column.SEQUENCE_COL); + colNameToExpression.remove(Column.SEQUENCE_COL); + colNameToExpression.remove(seqColumnName); + } else if (colNameToExpression.containsKey(seqColumnName)) { + forSeqCol = colNameToExpression.get(seqColumnName); + colNameToExpression.remove(seqColumnName); + } else { + forSeqCol = new Now(); + } + builder.add(new Cast(forSeqCol, seqColType.map(DataType::fromCatalogType).get())); + } + if (!colNameToExpression.isEmpty()) { + throw new AnalysisException("unknown column in target table: " + + String.join(", ", colNameToExpression.keySet())); + } + return builder.build(); + } + + private List<NamedExpression> generateFinalProjections(List<String> colNames, + List<List<Expression>> finalProjections) { + for (List<Expression> projection : finalProjections) { + if (projection.size() != finalProjections.get(0).size()) { + throw new AnalysisException("Column count doesn't match each other"); + } + } + ImmutableList.Builder<NamedExpression> outputProjectionsBuilder = ImmutableList.builder(); + for (int i = 0; i < finalProjections.get(0).size(); i++) { + Expression project = new NullLiteral(); + for (int j = 0; j < finalProjections.size(); j++) { + project = new If(new EqualTo(new UnboundSlot(BRANCH_LABEL), new IntegerLiteral(j)), + finalProjections.get(j).get(i), project); + } + outputProjectionsBuilder.add(new UnboundAlias(project, colNames.get(i))); + } + return outputProjectionsBuilder.build(); + } + + /** + * complete merge into plan. + */ + private LogicalPlan completeQueryPlan(ConnectContext ctx) { + // check insert include all keys + OlapTable targetTable = getTargetTable(ctx); + List<Column> columns = targetTable.getBaseSchema(true); + // compute sequence column info + boolean hasSequenceCol = targetTable.hasSequenceCol(); + String seqColName = null; + int seqColumnIndex = -1; + Optional<Column> seqMappingColInTable = Optional.empty(); + if (hasSequenceCol) { + seqColName = targetTable.getSequenceMapCol(); + String finalSeqColName = seqColName; + if (seqColName != null) { + for (int i = 0; i < columns.size(); i++) { + Column column = columns.get(i); + if (column.getName().equalsIgnoreCase(seqColName)) { + seqColumnIndex = i; + break; + } + } + seqMappingColInTable = columns.stream() + .filter(col -> col.getName().equalsIgnoreCase(finalSeqColName)) + .findFirst(); + } + + } + if (seqColumnIndex != -1 && !seqMappingColInTable.isPresent()) { + throw new AnalysisException("sequence column is not contained in" + + " target table " + targetTable.getName()); + } + + // generate base plan + LogicalPlan plan = generateBasePlan(); + // generate a project to add delete sign, seq column, label and mark + ImmutableList.Builder<NamedExpression> outputProjections = ImmutableList.builder(); + outputProjections.add(new UnboundStar(ImmutableList.of())); + List<String> targetDeleteSignNameParts = Lists.newArrayList(targetNameInPlan); + targetDeleteSignNameParts.add(Column.DELETE_SIGN); + NamedExpression deleteSign = new UnboundSlot(targetDeleteSignNameParts); + outputProjections.add(deleteSign); + outputProjections.add(generateBranchLabel(deleteSign)); + if (hasSequenceCol) { + List<String> targetSeqColNameParts = Lists.newArrayList(targetNameInPlan); + targetSeqColNameParts.add(Column.SEQUENCE_COL); + NamedExpression seqCol = new UnboundSlot(targetSeqColNameParts); + outputProjections.add(seqCol); + } + plan = new LogicalProject<>(outputProjections.build(), plan); + // remove all lines that do not be used for update, delete and insert + plan = new LogicalFilter<>(ImmutableSet.of(new Not(new IsNull(new UnboundSlot(BRANCH_LABEL)))), plan); + // compute final project by branch number and add delete sign + List<List<Expression>> finalProjections = Lists.newArrayList(); + // matched + for (MergeMatchedClause clause : matchedClauses) { + if (clause.isDelete()) { + finalProjections.add(generateDeleteProjection(columns)); + } else { + finalProjections.add(generateUpdateProjection(clause, columns, targetTable, ctx)); + } + } + // not matched + long columnCount = columns.stream().filter(Column::isVisible).count(); + for (MergeNotMatchedClause clause : notMatchedClauses) { + if (clause.getColNames().isEmpty()) { + if (columnCount != clause.getRow().size()) { + throw new AnalysisException("Column count doesn't match value count"); + } + finalProjections.add(generateInsertWithoutColListProjection(clause, columns, targetTable, + hasSequenceCol, seqColumnIndex, seqMappingColInTable, + Optional.ofNullable(targetTable.getSequenceType()))); + } else { + if (clause.getColNames().size() != clause.getRow().size()) { + throw new AnalysisException("Column count doesn't match value count"); + } + finalProjections.add(generateInsertWithColListProjection(clause, columns, targetTable, + hasSequenceCol, seqColName, seqMappingColInTable, + Optional.ofNullable(targetTable.getSequenceType()))); + } + } + List<String> colNames = columns.stream() + .filter(c -> (c.isVisible() && !c.isGeneratedColumn()) + || c.isDeleteSignColumn() || c.isSequenceColumn()) + .map(Column::getName) + .collect(ImmutableList.toImmutableList()); + plan = new LogicalProject<>(generateFinalProjections(colNames, finalProjections), plan); + + // TODO 6, 7, 8, 9 strict mode + // 6. add a set of new columns used for group by: if(mark = 1, target keys + mark, insert keys + mark) + // 7. add window node, partition by group by key, order by 1, row number, count(update) as uc, max(delete) as dc + // 8. get row_number = 1 + // 9. assert_true(uc <= 1 and (uc = 0 || dc = 0) (optional) + + if (cte.isPresent()) { + plan = (LogicalPlan) cte.get().withChildren(plan); + } + plan = UnboundTableSinkCreator.createUnboundTableSink(targetNameParts, colNames, ImmutableList.of(), + false, ImmutableList.of(), false, TPartialUpdateNewRowPolicy.APPEND, + DMLCommandType.INSERT, plan); + return plan; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeMatchedClause.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeMatchedClause.java new file mode 100644 index 00000000000..e7636dd0c46 --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeMatchedClause.java @@ -0,0 +1,55 @@ +// 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.merge; + +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.util.Utils; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * matched clause for merge into. + */ +public class MergeMatchedClause { + + private final Optional<Expression> casePredicate; + private final List<EqualTo> assignments; + private final boolean isDelete; + + public MergeMatchedClause(Optional<Expression> casePredicate, List<EqualTo> assignments, boolean isDelete) { + this.casePredicate = Objects.requireNonNull(casePredicate, "casePredicate should not be null"); + this.assignments = Utils.fastToImmutableList( + Objects.requireNonNull(assignments, "assignments should not be null")); + this.isDelete = isDelete; + } + + public Optional<Expression> getCasePredicate() { + return casePredicate; + } + + public List<EqualTo> getAssignments() { + return assignments; + } + + public boolean isDelete() { + return isDelete; + } +} diff --git a/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeNotMatchedClause.java b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeNotMatchedClause.java new file mode 100644 index 00000000000..4a091b1875d --- /dev/null +++ b/fe/fe-core/src/main/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeNotMatchedClause.java @@ -0,0 +1,55 @@ +// 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.merge; + +import org.apache.doris.nereids.trees.expressions.Expression; +import org.apache.doris.nereids.trees.expressions.NamedExpression; +import org.apache.doris.nereids.util.Utils; + +import java.util.List; +import java.util.Objects; +import java.util.Optional; + +/** + * not matched clause for merge into. + */ +public class MergeNotMatchedClause { + + private final Optional<Expression> casePredicate; + private final List<String> colNames; + private final List<NamedExpression> row; + + public MergeNotMatchedClause(Optional<Expression> casePredicate, + List<String> colNames, List<NamedExpression> row) { + this.casePredicate = Objects.requireNonNull(casePredicate, "casePredicate should not be null"); + this.colNames = Utils.fastToImmutableList(Objects.requireNonNull(colNames, "colNames should not be null")); + this.row = Utils.fastToImmutableList(Objects.requireNonNull(row, "row should not be null")); + } + + public Optional<Expression> getCasePredicate() { + return casePredicate; + } + + public List<String> getColNames() { + return colNames; + } + + public List<NamedExpression> getRow() { + return row; + } +} 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 b663110a5ca..374f5fbda7f 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 @@ -297,6 +297,7 @@ import org.apache.doris.nereids.trees.plans.commands.load.PauseRoutineLoadComman import org.apache.doris.nereids.trees.plans.commands.load.ResumeRoutineLoadCommand; import org.apache.doris.nereids.trees.plans.commands.load.ShowCreateRoutineLoadCommand; import org.apache.doris.nereids.trees.plans.commands.load.StopRoutineLoadCommand; +import org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshCatalogCommand; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshDatabaseCommand; import org.apache.doris.nereids.trees.plans.commands.refresh.RefreshDictionaryCommand; @@ -354,6 +355,10 @@ public interface CommandVisitor<R, C> { return visitCommand(deleteFromUsingCommand, context); } + default R visitMergeIntoCommand(MergeIntoCommand mergeIntoCommand, C context) { + return visitCommand(mergeIntoCommand, context); + } + default R visitLoadCommand(LoadCommand loadCommand, C context) { return visitCommand(loadCommand, context); } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java index 0de0b043f36..a6137e6c59d 100644 --- a/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/parser/NereidsParserTest.java @@ -46,6 +46,7 @@ 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.OptimizeTableCommand; import org.apache.doris.nereids.trees.plans.commands.ReplayCommand; +import org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand; 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.LogicalJoin; @@ -1301,4 +1302,97 @@ public class NereidsParserTest extends ParserTestBase { .assertThrowsExactly(SyntaxParseException.class) .assertMessageContains("mismatched input ')' expecting"); } + + @Test + public void testMergeInto() throws Exception { + NereidsParser parser = new NereidsParser(); + String sql; + LogicalPlan logicalPlan; + + // base case + sql = "MERGE INTO target USING source ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT VALUES (c1, c2, c3)"; + logicalPlan = parser.parseSingle(sql); + Assertions.assertInstanceOf(MergeIntoCommand.class, logicalPlan); + + // base case + target alias + sql = "MERGE INTO target target USING source ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT VALUES (c1, c2, c3)"; + logicalPlan = parser.parseSingle(sql); + Assertions.assertInstanceOf(MergeIntoCommand.class, logicalPlan); + + // base case + target alias with as + sql = "MERGE INTO target AS target USING source ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT VALUES (c1, c2, c3)"; + logicalPlan = parser.parseSingle(sql); + Assertions.assertInstanceOf(MergeIntoCommand.class, logicalPlan); + + // base case + insert column list + sql = "MERGE INTO target USING source ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT (c1, c2, c3) VALUES (c1, c2, c3)"; + logicalPlan = parser.parseSingle(sql); + Assertions.assertInstanceOf(MergeIntoCommand.class, logicalPlan); + + // base case + without not matched + sql = "MERGE INTO target USING source ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE "; + logicalPlan = parser.parseSingle(sql); + Assertions.assertInstanceOf(MergeIntoCommand.class, logicalPlan); + + // base case + without delete matched + sql = "MERGE INTO target USING source ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN NOT MATCHED THEN INSERT (c1, c2, c3) VALUES (c1, c2, c3)"; + logicalPlan = parser.parseSingle(sql); + Assertions.assertInstanceOf(MergeIntoCommand.class, logicalPlan); + + // base case + without update matched + sql = "MERGE INTO target USING source ON target.c1 = source.c1 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT (c1, c2, c3) VALUES (c1, c2, c3)"; + logicalPlan = parser.parseSingle(sql); + Assertions.assertInstanceOf(MergeIntoCommand.class, logicalPlan); + + // base case + insert with case predicate + sql = "MERGE INTO target USING source ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED AND source.c1 < 10 THEN INSERT (c1, c2, c3) VALUES (c1, c2, c3)"; + logicalPlan = parser.parseSingle(sql); + Assertions.assertInstanceOf(MergeIntoCommand.class, logicalPlan); + + // base case without and matched or not matched + String invalidSql1 = "MERGE INTO target USING source ON target.c1 = source.c1 "; + Assertions.assertThrows(ParseException.class, () -> parser.parseSingle(invalidSql1)); + + // base case without using + String invalidSql2 = "MERGE INTO target ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT VALUES (c1, c2, c3)"; + Assertions.assertThrows(ParseException.class, () -> parser.parseSingle(invalidSql2)); + + // base case without on clause + String invalidSql3 = "MERGE INTO target USING source " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT VALUES (c1, c2, c3)"; + Assertions.assertThrows(ParseException.class, () -> parser.parseSingle(invalidSql3)); + + // base case without target table + String invalidSql4 = "MERGE INTO USING source ON target.c1 = source.c1 " + + "WHEN MATCHED AND target.c2 > 5 THEN UPDATE SET c2 = c2 * 5 " + + "WHEN MATCHED THEN DELETE " + + "WHEN NOT MATCHED THEN INSERT VALUES (c1, c2, c3)"; + Assertions.assertThrows(ParseException.class, () -> parser.parseSingle(invalidSql4)); + } } diff --git a/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeIntoCommandTest.java b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeIntoCommandTest.java new file mode 100644 index 00000000000..f6be2b3d46b --- /dev/null +++ b/fe/fe-core/src/test/java/org/apache/doris/nereids/trees/plans/commands/merge/MergeIntoCommandTest.java @@ -0,0 +1,959 @@ +// 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.merge; + +import org.apache.doris.analysis.DefaultValueExprDef; +import org.apache.doris.analysis.IntLiteral; +import org.apache.doris.catalog.AggregateType; +import org.apache.doris.catalog.Column; +import org.apache.doris.catalog.GeneratedColumnInfo; +import org.apache.doris.catalog.OlapTable; +import org.apache.doris.catalog.PrimitiveType; +import org.apache.doris.catalog.ScalarType; +import org.apache.doris.nereids.analyzer.UnboundAlias; +import org.apache.doris.nereids.analyzer.UnboundFunction; +import org.apache.doris.nereids.analyzer.UnboundRelation; +import org.apache.doris.nereids.analyzer.UnboundSlot; +import org.apache.doris.nereids.exceptions.AnalysisException; +import org.apache.doris.nereids.trees.expressions.Cast; +import org.apache.doris.nereids.trees.expressions.DefaultValueSlot; +import org.apache.doris.nereids.trees.expressions.EqualTo; +import org.apache.doris.nereids.trees.expressions.Expression; +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.functions.scalar.If; +import org.apache.doris.nereids.trees.expressions.literal.IntegerLiteral; +import org.apache.doris.nereids.trees.expressions.literal.NullLiteral; +import org.apache.doris.nereids.trees.expressions.literal.TinyIntLiteral; +import org.apache.doris.nereids.trees.plans.JoinType; +import org.apache.doris.nereids.trees.plans.RelationId; +import org.apache.doris.nereids.trees.plans.commands.UpdateCommand; +import org.apache.doris.nereids.trees.plans.logical.LogicalCheckPolicy; +import org.apache.doris.nereids.trees.plans.logical.LogicalEmptyRelation; +import org.apache.doris.nereids.trees.plans.logical.LogicalJoin; +import org.apache.doris.nereids.trees.plans.logical.LogicalPlan; +import org.apache.doris.nereids.trees.plans.logical.LogicalSubQueryAlias; +import org.apache.doris.nereids.types.BigIntType; +import org.apache.doris.nereids.types.DateV2Type; +import org.apache.doris.qe.ConnectContext; + +import com.google.common.collect.ImmutableList; +import org.junit.jupiter.api.Assertions; +import org.junit.jupiter.api.Test; +import org.mockito.MockedStatic; +import org.mockito.Mockito; + +import java.lang.reflect.InvocationTargetException; +import java.lang.reflect.Method; +import java.util.HashSet; +import java.util.List; +import java.util.Optional; + +public class MergeIntoCommandTest { + + @Test + public void testGenerateBasePlanWithAlias() throws Exception { + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, new NullLiteral(), + ImmutableList.of(), ImmutableList.of() + ); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateBasePlan = clazz.getDeclaredMethod("generateBasePlan"); + generateBasePlan.setAccessible(true); + LogicalPlan result = (LogicalPlan) generateBasePlan.invoke(command); + Assertions.assertInstanceOf(LogicalJoin.class, result); + LogicalJoin<?, ?> logicalJoin = (LogicalJoin<?, ?>) result; + Assertions.assertEquals(1, logicalJoin.getOtherJoinConjuncts().size()); + Expression onClause = logicalJoin.getOtherJoinConjuncts().get(0); + Assertions.assertEquals(new NullLiteral(), onClause); + Assertions.assertEquals(JoinType.LEFT_OUTER_JOIN, logicalJoin.getJoinType()); + Assertions.assertEquals(source, logicalJoin.left()); + Assertions.assertInstanceOf(LogicalSubQueryAlias.class, logicalJoin.right()); + LogicalSubQueryAlias<?> alias = (LogicalSubQueryAlias<?>) logicalJoin.right(); + Assertions.assertEquals("alias", alias.getAlias()); + } + + @Test + public void testGenerateBasePlanWithoutAlias() throws Exception { + List<String> nameParts = ImmutableList.of("ctl", "db", "tbl"); + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + MergeIntoCommand command = new MergeIntoCommand( + nameParts, Optional.empty(), Optional.empty(), + source, new NullLiteral(), + ImmutableList.of(), ImmutableList.of() + ); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateBasePlan = clazz.getDeclaredMethod("generateBasePlan"); + generateBasePlan.setAccessible(true); + LogicalPlan result = (LogicalPlan) generateBasePlan.invoke(command); + Assertions.assertInstanceOf(LogicalJoin.class, result); + LogicalJoin<?, ?> logicalJoin = (LogicalJoin<?, ?>) result; + Assertions.assertEquals(1, logicalJoin.getOtherJoinConjuncts().size()); + Expression onClause = logicalJoin.getOtherJoinConjuncts().get(0); + Assertions.assertEquals(new NullLiteral(), onClause); + Assertions.assertEquals(JoinType.LEFT_OUTER_JOIN, logicalJoin.getJoinType()); + Assertions.assertEquals(source, logicalJoin.left()); + Assertions.assertInstanceOf(LogicalCheckPolicy.class, logicalJoin.right()); + Assertions.assertInstanceOf(UnboundRelation.class, logicalJoin.right().child(0)); + UnboundRelation unboundRelation = (UnboundRelation) logicalJoin.right().child(0); + Assertions.assertEquals(nameParts, unboundRelation.getNameParts()); + } + + @Test + public void testGenerateBranchLabel() throws Exception { + List<MergeMatchedClause> matchedClauses = ImmutableList.of( + new MergeMatchedClause(Optional.of(new IntegerLiteral(1)), ImmutableList.of(), true), + new MergeMatchedClause(Optional.of(new IntegerLiteral(2)), ImmutableList.of(), true), + new MergeMatchedClause(Optional.empty(), ImmutableList.of(), true) + ); + List<MergeNotMatchedClause> notMatchedClauses = ImmutableList.of( + new MergeNotMatchedClause(Optional.of(new IntegerLiteral(3)), ImmutableList.of(), ImmutableList.of()), + new MergeNotMatchedClause(Optional.of(new IntegerLiteral(4)), ImmutableList.of(), ImmutableList.of()), + new MergeNotMatchedClause(Optional.empty(), ImmutableList.of(), ImmutableList.of()) + ); + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, matchedClauses, notMatchedClauses); + UnboundSlot unboundSlot = new UnboundSlot("alias", "__DORIS_DELETE_SIGN__"); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateBranchLabel = clazz.getDeclaredMethod("generateBranchLabel", NamedExpression.class); + generateBranchLabel.setAccessible(true); + NamedExpression result = (NamedExpression) generateBranchLabel.invoke(command, unboundSlot); + Expression matchedLabel = new If(new IntegerLiteral(1), new IntegerLiteral(0), + new If(new IntegerLiteral(2), new IntegerLiteral(1), new IntegerLiteral(2))); + Expression notMatchedLabel = new If(new IntegerLiteral(3), new IntegerLiteral(3), + new If(new IntegerLiteral(4), new IntegerLiteral(4), new IntegerLiteral(5))); + NamedExpression expected = new UnboundAlias(new If(new Not(new IsNull(unboundSlot)), + matchedLabel, notMatchedLabel), "__DORIS_MERGE_INTO_BRANCH_LABEL__"); + Assertions.assertEquals(expected, result); + } + + @Test + public void testGenerateDeleteProjection() throws Exception { + List<Column> columns = ImmutableList.of( + new Column("c1", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateDeleteProjection = clazz.getDeclaredMethod("generateDeleteProjection", List.class); + generateDeleteProjection.setAccessible(true); + List<Expression> result = (List<Expression>) generateDeleteProjection.invoke(command, columns); + Assertions.assertEquals(4, result.size()); + Assertions.assertEquals(ImmutableList.of( + new UnboundSlot(ImmutableList.of("alias", "c1")), + new TinyIntLiteral(((byte) 1)), + new UnboundSlot(ImmutableList.of("alias", Column.SEQUENCE_COL)), + new UnboundSlot(ImmutableList.of("alias", "c3"))), result); + } + + @Test + public void testGenerateUpdateProjection() throws Exception { + MergeMatchedClause mergeMatchedClause = new MergeMatchedClause(Optional.empty(), ImmutableList.of( + new EqualTo(new UnboundSlot(Column.SEQUENCE_COL), new IntegerLiteral(1)), + new EqualTo(new UnboundSlot("c4"), new IntegerLiteral(2)), + new EqualTo(new UnboundSlot("c5"), new IntegerLiteral(3)) + ), false); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column("c4", PrimitiveType.BIGINT), + new Column("c5", PrimitiveType.BIGINT), + new Column("c6", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", true, new DefaultValueExprDef("CURRENT_TIMESTAMP"), 1, + null, new HashSet<>()) + ); + + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateUpdateProjection = clazz.getDeclaredMethod("generateUpdateProjection", + MergeMatchedClause.class, List.class, OlapTable.class, ConnectContext.class); + generateUpdateProjection.setAccessible(true); + + try (MockedStatic<UpdateCommand> mockedUpdate = Mockito.mockStatic(UpdateCommand.class)) { + mockedUpdate.when(() -> UpdateCommand.checkAssignmentColumn(Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any())).thenAnswer(x -> null); + List<Expression> result = (List<Expression>) generateUpdateProjection.invoke(command, + mergeMatchedClause, columns, olapTable, null); + List<Expression> expected = ImmutableList.of( + new Cast(new UnboundSlot(ImmutableList.of("alias", "c1")), BigIntType.INSTANCE), + new Cast(new UnboundSlot(ImmutableList.of("alias", Column.DELETE_SIGN)), BigIntType.INSTANCE), + new Cast(new IntegerLiteral(1), BigIntType.INSTANCE), + new Cast(new UnboundSlot(ImmutableList.of("alias", "c3")), BigIntType.INSTANCE), + new Cast(new IntegerLiteral(2), BigIntType.INSTANCE), + new Cast(new IntegerLiteral(3), BigIntType.INSTANCE), + new Cast(new UnboundFunction("current_timestamp", ImmutableList.of()), BigIntType.INSTANCE) + ); + Assertions.assertEquals(expected, result); + } + } + + @Test + public void testGenerateUpdateProjectionWithDuplicateColumn() throws Exception { + MergeMatchedClause mergeMatchedClause = new MergeMatchedClause(Optional.empty(), ImmutableList.of( + new EqualTo(new UnboundSlot(Column.SEQUENCE_COL), new IntegerLiteral(1)), + new EqualTo(new UnboundSlot("c4"), new IntegerLiteral(2)), + new EqualTo(new UnboundSlot("c4"), new IntegerLiteral(3)) + ), false); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column("c4", PrimitiveType.BIGINT), + new Column("c5", PrimitiveType.BIGINT), + new Column("c6", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", true, new DefaultValueExprDef("CURRENT_TIMESTAMP"), 1, + null, new HashSet<>()) + ); + + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateUpdateProjection = clazz.getDeclaredMethod("generateUpdateProjection", + MergeMatchedClause.class, List.class, OlapTable.class, ConnectContext.class); + generateUpdateProjection.setAccessible(true); + + try (MockedStatic<UpdateCommand> mockedUpdate = Mockito.mockStatic(UpdateCommand.class)) { + mockedUpdate.when(() -> UpdateCommand.checkAssignmentColumn(Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any())).thenAnswer(x -> null); + try { + generateUpdateProjection.invoke(command, mergeMatchedClause, columns, olapTable, null); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + } + + @Test + public void testGenerateUpdateProjectionWithKey() throws Exception { + MergeMatchedClause mergeMatchedClause = new MergeMatchedClause(Optional.empty(), ImmutableList.of( + new EqualTo(new UnboundSlot(Column.SEQUENCE_COL), new IntegerLiteral(1)), + new EqualTo(new UnboundSlot("c1"), new IntegerLiteral(2)), + new EqualTo(new UnboundSlot("c5"), new IntegerLiteral(3)) + ), false); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column("c4", PrimitiveType.BIGINT), + new Column("c5", PrimitiveType.BIGINT), + new Column("c6", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", true, new DefaultValueExprDef("CURRENT_TIMESTAMP"), 1, + null, new HashSet<>()) + ); + + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateUpdateProjection = clazz.getDeclaredMethod("generateUpdateProjection", + MergeMatchedClause.class, List.class, OlapTable.class, ConnectContext.class); + generateUpdateProjection.setAccessible(true); + + try (MockedStatic<UpdateCommand> mockedUpdate = Mockito.mockStatic(UpdateCommand.class)) { + mockedUpdate.when(() -> UpdateCommand.checkAssignmentColumn(Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any())).thenAnswer(x -> null); + try { + generateUpdateProjection.invoke(command, mergeMatchedClause, columns, olapTable, null); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + } + + @Test + public void testGenerateUpdateProjectionWithGeneratedColumn() throws Exception { + MergeMatchedClause mergeMatchedClause = new MergeMatchedClause(Optional.empty(), ImmutableList.of( + new EqualTo(new UnboundSlot(Column.SEQUENCE_COL), new IntegerLiteral(1)), + new EqualTo(new UnboundSlot("c2"), new IntegerLiteral(2)), + new EqualTo(new UnboundSlot("c5"), new IntegerLiteral(3)) + ), false); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column("c4", PrimitiveType.BIGINT), + new Column("c5", PrimitiveType.BIGINT), + new Column("c6", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", true, new DefaultValueExprDef("CURRENT_TIMESTAMP"), 1, + null, new HashSet<>()) + ); + + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateUpdateProjection = clazz.getDeclaredMethod("generateUpdateProjection", + MergeMatchedClause.class, List.class, OlapTable.class, ConnectContext.class); + generateUpdateProjection.setAccessible(true); + + try (MockedStatic<UpdateCommand> mockedUpdate = Mockito.mockStatic(UpdateCommand.class)) { + mockedUpdate.when(() -> UpdateCommand.checkAssignmentColumn(Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any())).thenAnswer(x -> null); + try { + generateUpdateProjection.invoke(command, mergeMatchedClause, columns, olapTable, null); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + } + + @Test + public void testGenerateUpdateProjectionWithNonExistsColumn() throws Exception { + MergeMatchedClause mergeMatchedClause = new MergeMatchedClause(Optional.empty(), ImmutableList.of( + new EqualTo(new UnboundSlot(Column.SEQUENCE_COL), new IntegerLiteral(1)), + new EqualTo(new UnboundSlot("c1"), new IntegerLiteral(2)), + new EqualTo(new UnboundSlot("c10"), new IntegerLiteral(3)) + ), false); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column("c4", PrimitiveType.BIGINT), + new Column("c5", PrimitiveType.BIGINT), + new Column("c6", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", true, new DefaultValueExprDef("CURRENT_TIMESTAMP"), 1, + null, new HashSet<>()) + ); + + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateUpdateProjection = clazz.getDeclaredMethod("generateUpdateProjection", + MergeMatchedClause.class, List.class, OlapTable.class, ConnectContext.class); + generateUpdateProjection.setAccessible(true); + + try (MockedStatic<UpdateCommand> mockedUpdate = Mockito.mockStatic(UpdateCommand.class)) { + mockedUpdate.when(() -> UpdateCommand.checkAssignmentColumn(Mockito.any(), Mockito.any(), + Mockito.any(), Mockito.any())).thenAnswer(x -> null); + try { + generateUpdateProjection.invoke(command, mergeMatchedClause, columns, olapTable, null); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + } + + @Test + public void testGenerateInsertWithoutColListProjectionWithoutSeqCol() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of(), + ImmutableList.of(new UnboundSlot("c1"), new UnboundAlias(new DefaultValueSlot()), new UnboundSlot("c3"))); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithoutColListProjection = clazz.getDeclaredMethod("generateInsertWithoutColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, int.class, Optional.class, Optional.class); + generateInsertWithoutColListProjection.setAccessible(true); + List<Expression> result = (List<Expression>) generateInsertWithoutColListProjection.invoke(command, clause, columns, olapTable, false, -1, Optional.empty(), Optional.empty()); + List<Expression> expected = ImmutableList.of( + new Cast(new UnboundSlot("c1"), BigIntType.INSTANCE), + new Cast(new UnboundSlot("c3"), BigIntType.INSTANCE), + new TinyIntLiteral(((byte) 0)) + ); + Assertions.assertEquals(expected, result); + } + + @Test + public void testGenerateInsertWithoutColListProjectionWithSeqColumnWithSeqIndex() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of(), + ImmutableList.of(new UnboundSlot("c1"), new UnboundAlias(new DefaultValueSlot()), new UnboundSlot("c3"))); + + Column seqCol = new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, -1, "default", "", + true, null, -1, "", true, new DefaultValueExprDef("CURRENT_TIMESTAMP"), 1, + null, new HashSet<>()); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithoutColListProjection = clazz.getDeclaredMethod("generateInsertWithoutColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, int.class, Optional.class, Optional.class); + generateInsertWithoutColListProjection.setAccessible(true); + List<Expression> result = (List<Expression>) generateInsertWithoutColListProjection.invoke(command, clause, columns, olapTable, true, 2, Optional.of(seqCol), Optional.of(ScalarType.createType(PrimitiveType.BIGINT))); + List<Expression> expected = ImmutableList.of( + new Cast(new UnboundSlot("c1"), BigIntType.INSTANCE), + new Cast(new UnboundSlot("c3"), BigIntType.INSTANCE), + new TinyIntLiteral(((byte) 0)), + new Cast(new Cast(new UnboundSlot("c3"), BigIntType.INSTANCE), BigIntType.INSTANCE) + ); + Assertions.assertEquals(expected, result); + } + + @Test + public void testGenerateInsertWithoutColListProjectionWithSeqColumnWithSeqIndexWithoutDefaultValue() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of(), + ImmutableList.of(new UnboundSlot("c1"), new UnboundAlias(new DefaultValueSlot()), new UnboundSlot("c3"))); + + Column seqCol = new Column("c3", PrimitiveType.BIGINT); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithoutColListProjection = clazz.getDeclaredMethod("generateInsertWithoutColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, int.class, Optional.class, Optional.class); + generateInsertWithoutColListProjection.setAccessible(true); + List<Expression> result = (List<Expression>) generateInsertWithoutColListProjection.invoke(command, clause, columns, olapTable, true, 2, Optional.of(seqCol), Optional.of(ScalarType.createType(PrimitiveType.BIGINT))); + List<Expression> expected = ImmutableList.of( + new Cast(new UnboundSlot("c1"), BigIntType.INSTANCE), + new Cast(new UnboundSlot("c3"), BigIntType.INSTANCE), + new TinyIntLiteral(((byte) 0)), + new Cast(new Cast(new UnboundSlot("c3"), BigIntType.INSTANCE), BigIntType.INSTANCE) + ); + Assertions.assertEquals(expected, result); + } + + @Test + public void testGenerateInsertWithoutColListProjectionWithSeqColumnWithoutSeqIndex() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1), + new Column(Column.VERSION_COL, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of(), + ImmutableList.of(new UnboundSlot("c1"), new UnboundAlias(new DefaultValueSlot()), new UnboundSlot("c3"))); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithoutColListProjection = clazz.getDeclaredMethod("generateInsertWithoutColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, int.class, Optional.class, Optional.class); + generateInsertWithoutColListProjection.setAccessible(true); + try { + generateInsertWithoutColListProjection.invoke(command, clause, columns, olapTable, true, -1, Optional.empty(), Optional.empty()); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + + @Test + public void testGenerateInsertWithoutColListProjectionWithGeneratedColumnWithoutDefaultValue() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of(), + ImmutableList.of(new UnboundSlot("c1"), new UnboundSlot("c2"), new UnboundSlot("c3"))); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithoutColListProjection = clazz.getDeclaredMethod("generateInsertWithoutColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, int.class, Optional.class, Optional.class); + generateInsertWithoutColListProjection.setAccessible(true); + try { + generateInsertWithoutColListProjection.invoke(command, clause, columns, olapTable, false, -1, Optional.empty(), Optional.empty()); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + + @Test + public void testGenerateInsertWithColListProjectionWithoutSeqCol() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of("c1", "c2", "c3"), + ImmutableList.of(new UnboundSlot("c1"), new UnboundAlias(new DefaultValueSlot()), new UnboundSlot("c3"))); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + // generated column + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, -1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column("c3", PrimitiveType.BIGINT), + // auto inc + new Column("c4", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, 1, null, "", + true, null, -1, "", false, null, 1, + null, new HashSet<>()), + // null default + new Column("c5", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, true, null, ""), + // default expr + new Column("c6", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "1", ""), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithColListProjection = clazz.getDeclaredMethod("generateInsertWithColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, String.class, Optional.class, Optional.class); + generateInsertWithColListProjection.setAccessible(true); + List<Expression> result = (List<Expression>) generateInsertWithColListProjection.invoke(command, clause, columns, olapTable, false, "", Optional.empty(), Optional.empty()); + List<Expression> expected = ImmutableList.of( + new Cast(new UnboundSlot("c1"), BigIntType.INSTANCE), + new Cast(new UnboundSlot("c3"), BigIntType.INSTANCE), + new NullLiteral(BigIntType.INSTANCE), + new NullLiteral(BigIntType.INSTANCE), + new Cast(new TinyIntLiteral((byte) 1), BigIntType.INSTANCE), + new TinyIntLiteral(((byte) 0)) + ); + Assertions.assertEquals(expected, result); + } + + @Test + public void testGenerateInsertWithColListProjectionWithSeqColWithSeqIndex() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of("c1", "c3"), + ImmutableList.of(new UnboundSlot("c1"), new UnboundSlot("c3"))); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c3", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + Column seqCol = new Column(Column.SEQUENCE_COL, ScalarType.createType(PrimitiveType.DATEV2), false, + AggregateType.NONE, false, -1, null, "", + true, null, -1, "", false, null, 1, + null, new HashSet<>()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithColListProjection = clazz.getDeclaredMethod("generateInsertWithColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, String.class, Optional.class, Optional.class); + generateInsertWithColListProjection.setAccessible(true); + List<Expression> result = (List<Expression>) generateInsertWithColListProjection.invoke(command, clause, columns, olapTable, true, "c3", Optional.of(seqCol), Optional.of(ScalarType.createType(PrimitiveType.DATEV2))); + List<Expression> expected = ImmutableList.of( + new Cast(new UnboundSlot("c1"), BigIntType.INSTANCE), + new Cast(new UnboundSlot("c3"), BigIntType.INSTANCE), + new TinyIntLiteral(((byte) 0)), + new Cast(new UnboundSlot("c3"), DateV2Type.INSTANCE) + ); + Assertions.assertEquals(expected, result); + } + + @Test + public void testGenerateInsertWithColListProjectionWithSeqColWithoutSeqIndexWithoutDefaultValue() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of("c1", "c3"), + ImmutableList.of(new UnboundSlot("c1"), new UnboundSlot("c3"))); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c3", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithColListProjection = clazz.getDeclaredMethod("generateInsertWithColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, String.class, Optional.class, Optional.class); + generateInsertWithColListProjection.setAccessible(true); + try { + generateInsertWithColListProjection.invoke(command, clause, columns, olapTable, true, "", Optional.empty(), Optional.empty()); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + + @Test + public void testGenerateInsertWithColListProjectionWithDuplicateColLabel() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of("c1", "c1"), + ImmutableList.of(new UnboundSlot("c1"), new UnboundSlot("c3"))); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c3", PrimitiveType.BIGINT), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithColListProjection = clazz.getDeclaredMethod("generateInsertWithColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, String.class, Optional.class, Optional.class); + generateInsertWithColListProjection.setAccessible(true); + try { + generateInsertWithColListProjection.invoke(command, clause, columns, olapTable, false, "", Optional.empty(), Optional.empty()); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + + @Test + public void testGenerateInsertWithColListProjectionWithGeneratedCol() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of("c1", "c2"), + ImmutableList.of(new UnboundSlot("c1"), new UnboundSlot("c2"))); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c2", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, -1, "default", "", + true, null, -1, "", false, null, 1, + new GeneratedColumnInfo("cc", new IntLiteral(1)), new HashSet<>()), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithColListProjection = clazz.getDeclaredMethod("generateInsertWithColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, String.class, Optional.class, Optional.class); + generateInsertWithColListProjection.setAccessible(true); + try { + generateInsertWithColListProjection.invoke(command, clause, columns, olapTable, false, "", Optional.empty(), Optional.empty()); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + + @Test + public void testGenerateInsertWithColListProjectionWithNotNullWithoutDefaultValue() throws Exception { + OlapTable olapTable = Mockito.mock(OlapTable.class); + Mockito.when(olapTable.getName()).thenReturn("olap_table"); + + MergeNotMatchedClause clause = new MergeNotMatchedClause(Optional.empty(), ImmutableList.of("c1", "c3"), + ImmutableList.of(new UnboundSlot("c1"), new UnboundSlot("c3"))); + + List<Column> columns = ImmutableList.of( + new Column("c1", ScalarType.createType(PrimitiveType.BIGINT), true, AggregateType.NONE, "", ""), + new Column("c3", PrimitiveType.BIGINT), + new Column("c5", ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, null, ""), + new Column(Column.DELETE_SIGN, ScalarType.createType(PrimitiveType.BIGINT), false, + AggregateType.NONE, false, "", false, -1) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateInsertWithColListProjection = clazz.getDeclaredMethod("generateInsertWithColListProjection", + MergeNotMatchedClause.class, List.class, OlapTable.class, boolean.class, String.class, Optional.class, Optional.class); + generateInsertWithColListProjection.setAccessible(true); + try { + generateInsertWithColListProjection.invoke(command, clause, columns, olapTable, false, "", Optional.empty(), Optional.empty()); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } + + @Test + public void testGenerateFinalProjections() throws Exception { + List<String> colNames = ImmutableList.of("c1", "c2"); + List<List<Expression>> finalProjections = ImmutableList.of( + ImmutableList.of(new IntegerLiteral(11), new IntegerLiteral(12)), + ImmutableList.of(new IntegerLiteral(21), new IntegerLiteral(22)), + ImmutableList.of(new IntegerLiteral(31), new IntegerLiteral(32)), + ImmutableList.of(new IntegerLiteral(41), new IntegerLiteral(42)) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateFinalProjections = clazz.getDeclaredMethod("generateFinalProjections", List.class, List.class); + generateFinalProjections.setAccessible(true); + List<NamedExpression> result = (List<NamedExpression>) generateFinalProjections.invoke(command, colNames, finalProjections); + List<NamedExpression> expected = ImmutableList.of( + new UnboundAlias(new If(new EqualTo(new UnboundSlot("__DORIS_MERGE_INTO_BRANCH_LABEL__"), new IntegerLiteral(3)), new IntegerLiteral(41), + new If(new EqualTo(new UnboundSlot("__DORIS_MERGE_INTO_BRANCH_LABEL__"), new IntegerLiteral(2)), new IntegerLiteral(31), + new If(new EqualTo(new UnboundSlot("__DORIS_MERGE_INTO_BRANCH_LABEL__"), new IntegerLiteral(1)), new IntegerLiteral(21), + new If(new EqualTo(new UnboundSlot("__DORIS_MERGE_INTO_BRANCH_LABEL__"), new IntegerLiteral(0)), new IntegerLiteral(11), new NullLiteral())))), "c1"), + new UnboundAlias(new If(new EqualTo(new UnboundSlot("__DORIS_MERGE_INTO_BRANCH_LABEL__"), new IntegerLiteral(3)), new IntegerLiteral(42), + new If(new EqualTo(new UnboundSlot("__DORIS_MERGE_INTO_BRANCH_LABEL__"), new IntegerLiteral(2)), new IntegerLiteral(32), + new If(new EqualTo(new UnboundSlot("__DORIS_MERGE_INTO_BRANCH_LABEL__"), new IntegerLiteral(1)), new IntegerLiteral(22), + new If(new EqualTo(new UnboundSlot("__DORIS_MERGE_INTO_BRANCH_LABEL__"), new IntegerLiteral(0)), new IntegerLiteral(12), new NullLiteral())))), "c2") + ); + + Assertions.assertEquals(expected, result); + } + + @Test + public void testGenerateFinalProjectionsWithDiffSize() throws Exception { + List<String> colNames = ImmutableList.of("c1", "c2", "c3"); + List<List<Expression>> finalProjections = ImmutableList.of( + ImmutableList.of(new IntegerLiteral(11), new IntegerLiteral(12)), + ImmutableList.of(new IntegerLiteral(21), new IntegerLiteral(22)), + ImmutableList.of(new IntegerLiteral(31), new IntegerLiteral(32)), + ImmutableList.of(new IntegerLiteral(41), new IntegerLiteral(42)) + ); + + LogicalPlan source = new LogicalEmptyRelation(new RelationId(1), ImmutableList.of()); + Expression onClause = new NullLiteral(); + MergeIntoCommand command = new MergeIntoCommand( + ImmutableList.of("ctl", "db", "tbl"), Optional.of("alias"), Optional.empty(), + source, onClause, ImmutableList.of(), ImmutableList.of()); + + Class<?> clazz = Class.forName("org.apache.doris.nereids.trees.plans.commands.merge.MergeIntoCommand"); + Method generateFinalProjections = clazz.getDeclaredMethod("generateFinalProjections", List.class, List.class); + generateFinalProjections.setAccessible(true); + try { + generateFinalProjections.invoke(command, colNames, finalProjections); + } catch (InvocationTargetException e) { + Assertions.assertInstanceOf(AnalysisException.class, e.getCause()); + } + } +} diff --git a/regression-test/data/load_p0/merge_into/test_merge_into.out b/regression-test/data/load_p0/merge_into/test_merge_into.out new file mode 100644 index 00000000000..fac504ffd17 --- /dev/null +++ b/regression-test/data/load_p0/merge_into/test_merge_into.out @@ -0,0 +1,21 @@ +-- This file is automatically generated. You should know what you did if you want to edit this +-- !base_1 -- +\N \N +1 10 +3 33 + +-- !seq_col_1 -- +\N \N +1 10 +3 33 + +-- !seq_map_1 -- +\N \N 2025-02-02 +1 10 2020-02-02 +3 33 2025-02-02 + +-- !gen_col_1 -- +\N \N \N +1 10 11 +3 33 36 + diff --git a/regression-test/suites/load_p0/merge_into/test_merge_into.groovy b/regression-test/suites/load_p0/merge_into/test_merge_into.groovy new file mode 100644 index 00000000000..0422be5954d --- /dev/null +++ b/regression-test/suites/load_p0/merge_into/test_merge_into.groovy @@ -0,0 +1,254 @@ +// 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("test_merge_into") { + multi_sql """ + DROP TABLE IF EXISTS merge_into_source_table FORCE; + DROP TABLE IF EXISTS merge_into_target_base_table FORCE; + DROP TABLE IF EXISTS merge_into_target_seq_col_table FORCE; + DROP TABLE IF EXISTS merge_into_target_seq_map_table FORCE; + DROP TABLE IF EXISTS merge_into_target_gen_col_table FORCE; + + CREATE TABLE `merge_into_source_table` ( + `c1` int NULL, + `c2` varchar(255) NULL + ) ENGINE=OLAP + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + CREATE TABLE `merge_into_target_base_table` ( + `c1` int NULL, + `c2` varchar(255) NULL + ) ENGINE=OLAP + UNIQUE KEY(`c1`) + DISTRIBUTED BY HASH(`c1`) + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + CREATE TABLE `merge_into_target_seq_col_table` ( + `c1` int NULL, + `c2` varchar(255) NULL + ) ENGINE=OLAP + UNIQUE KEY(`c1`) + DISTRIBUTED BY HASH(`c1`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "function_column.sequence_type" = "date" + ); + + CREATE TABLE `merge_into_target_seq_map_table` ( + `c1` int NULL, + `c2` varchar(255) NULL, + `c3` date NULL + ) ENGINE=OLAP + UNIQUE KEY(`c1`) + DISTRIBUTED BY HASH(`c1`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1", + "function_column.sequence_col" = "c3" + ); + + CREATE TABLE `merge_into_target_gen_col_table` ( + `c1` int NULL, + `c2` int NULL, + `c3` bigint AS (c1 + c2) NULL + ) ENGINE=OLAP + UNIQUE KEY(`c1`) + DISTRIBUTED BY HASH(`c1`) BUCKETS 10 + PROPERTIES ( + "replication_allocation" = "tag.location.default: 1" + ); + + INSERT INTO merge_into_source_table VALUES (1, 12), (2, 22), (3, 33), (null, null); + INSERT INTO merge_into_target_base_table VALUES (1, 1), (2, 10); + INSERT INTO merge_into_target_seq_col_table (c1, c2, __DORIS_SEQUENCE_COL__) VALUES (1, 1, '2020-02-02'), (2, 10, '2020-02-02'); + INSERT INTO merge_into_target_seq_map_table VALUES (1, 1, '2020-02-02'), (2, 10, '2020-02-02'); + INSERT INTO merge_into_target_gen_col_table (c1, c2) VALUES (1, 1), (2, 10); + + SYNC; + """ + + // base merge, test null safe equals here + sql """ + WITH tmp AS (SELECT * FROM merge_into_source_table) + MERGE INTO merge_into_target_base_table t1 + USING tmp t2 + ON t1.c1 <=> t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN NOT MATCHED THEN INSERT VALUES(t2.c1, t2.c2) + """ + + sql """ + MERGE INTO merge_into_target_seq_col_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN NOT MATCHED THEN INSERT (c1, c2, __DORIS_SEQUENCE_COL__) VALUES(t2.c1, t2.c2, '2025-02-02') + """ + + sql """ + MERGE INTO merge_into_target_seq_map_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN NOT MATCHED THEN INSERT VALUES(t2.c1, t2.c2, '2025-02-02') + """ + + sql """ + MERGE INTO merge_into_target_gen_col_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN NOT MATCHED THEN INSERT (c1, c2) VALUES(t2.c1, t2.c2) + """ + + sql """ + SYNC + """ + + order_qt_base_1 """ + SELECT * FROM merge_into_target_base_table; + """ + order_qt_seq_col_1 """ + SELECT * FROM merge_into_target_seq_col_table; + """ + order_qt_seq_map_1 """ + SELECT * FROM merge_into_target_seq_map_table; + """ + order_qt_gen_col_1 """ + SELECT * FROM merge_into_target_gen_col_table; + """ + + // target has seq col but insert without seq col + test { + sql """ + MERGE INTO merge_into_target_seq_col_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN NOT MATCHED THEN INSERT (c1, c2) VALUES(t2.c1, t2.c2) + """ + + exception """has sequence column, need to specify the sequence column""" + } + + // target has generated col, update try to update it + test { + sql """ + MERGE INTO merge_into_target_gen_col_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10, c3 = 10 + WHEN NOT MATCHED THEN INSERT (c1, c2) VALUES(t2.c1, t2.c2) + """ + + exception """The value specified for generated column 'c3'""" + } + + // target has generated col, insert try to insert it explicitly + test { + sql """ + MERGE INTO merge_into_target_gen_col_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN NOT MATCHED THEN INSERT (c1, c2, c3) VALUES(t2.c1, t2.c2, t2.c2) + """ + + exception """The value specified for generated column 'c3'""" + } + + // update key column + test { + sql """ + MERGE INTO merge_into_target_base_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c1 = 10 + WHEN NOT MATCHED THEN INSERT VALUES(t2.c1, t2.c2) + """ + + exception """Only value columns of unique table could be updated""" + } + + // update not exist column + test { + sql """ + MERGE INTO merge_into_target_base_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c4 = 10 + WHEN NOT MATCHED THEN INSERT VALUES(t2.c1, t2.c2) + """ + + exception """unknown column in assignment list: c4""" + } + + // insert not exist column + test { + sql """ + MERGE INTO merge_into_target_base_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN NOT MATCHED THEN INSERT (c1, c2, c3) VALUES(t2.c1, t2.c2, t2.c2) + """ + + exception """unknown column in target table: c3""" + } + + // matched clause without predicate in the middle + test { + sql """ + MERGE INTO merge_into_target_base_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN NOT MATCHED THEN INSERT VALUES(t2.c1, t2.c2) + """ + + exception """Only the last matched clause could without case predicate""" + } + + // not matched clause without predicate in the middle + test { + sql """ + MERGE INTO merge_into_target_base_table t1 + USING merge_into_source_table t2 + ON t1.c1 = t2.c1 + WHEN MATCHED AND t1.c2 = 10 THEN DELETE + WHEN MATCHED THEN UPDATE SET c2 = 10 + WHEN NOT MATCHED THEN INSERT VALUES(t2.c1, t2.c2) + WHEN NOT MATCHED THEN INSERT VALUES(t2.c1, t2.c2) + """ + + exception """Only the last not matched clause could without case predicate""" + } +} \ No newline at end of file --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
