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

dkuzmenko pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/hive.git


The following commit(s) were added to refs/heads/master by this push:
     new e48a77cb07d HIVE-28084: Iceberg: COW fix for Merge operation (Denys 
Kuzmenko, reviewed by Ayush Saxena)
e48a77cb07d is described below

commit e48a77cb07d0091a23cfa17f142d2de71d9c2717
Author: Denys Kuzmenko <denisk...@gmail.com>
AuthorDate: Wed Feb 28 11:09:52 2024 +0200

    HIVE-28084: Iceberg: COW fix for Merge operation (Denys Kuzmenko, reviewed 
by Ayush Saxena)
    
    Closes #5088
---
 .../merge_iceberg_copy_on_write_unpartitioned.q    |   5 +
 ...merge_iceberg_copy_on_write_unpartitioned.q.out | 351 +++++++++++++++++++++
 .../ql/parse/rewrite/CopyOnWriteMergeRewriter.java |   9 +-
 3 files changed, 361 insertions(+), 4 deletions(-)

diff --git 
a/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_unpartitioned.q
 
b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_unpartitioned.q
index 34ac6ffe978..371e4b5e312 100644
--- 
a/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_unpartitioned.q
+++ 
b/iceberg/iceberg-handler/src/test/queries/positive/merge_iceberg_copy_on_write_unpartitioned.q
@@ -11,6 +11,11 @@ insert into target_ice values (1, 'one', 50), (2, 'two', 
51), (111, 'one', 55),
 insert into source values (1, 'one', 50), (2, 'two', 51), (3, 'three', 52), 
(4, 'four', 53), (5, 'five', 54), (111, 'one', 55);
 
 -- merge
+explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when not matched then insert values (src.a, src.b, src.c);
+
 explain
 merge into target_ice as t using source src ON t.a = src.a
 when matched and t.a > 100 THEN DELETE
diff --git 
a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out
 
b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out
index 0a4ba96cea2..14a9fd4c52b 100644
--- 
a/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out
+++ 
b/iceberg/iceberg-handler/src/test/results/positive/merge_iceberg_copy_on_write_unpartitioned.q.out
@@ -45,6 +45,357 @@ POSTHOOK: Output: default@source
 POSTHOOK: Lineage: source.a SCRIPT []
 POSTHOOK: Lineage: source.b SCRIPT []
 POSTHOOK: Lineage: source.c SCRIPT []
+PREHOOK: query: explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when not matched then insert values (src.a, src.b, src.c)
+PREHOOK: type: QUERY
+PREHOOK: Input: default@source
+PREHOOK: Input: default@target_ice
+PREHOOK: Output: default@target_ice
+POSTHOOK: query: explain
+merge into target_ice as t using source src ON t.a = src.a
+when matched and t.a > 100 THEN DELETE
+when not matched then insert values (src.a, src.b, src.c)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@source
+POSTHOOK: Input: default@target_ice
+POSTHOOK: Output: default@target_ice
+STAGE DEPENDENCIES:
+  Stage-1 is a root stage
+  Stage-2 depends on stages: Stage-1
+  Stage-0 depends on stages: Stage-2
+  Stage-3 depends on stages: Stage-0
+
+STAGE PLANS:
+  Stage: Stage-1
+    Tez
+#### A masked pattern was here ####
+      Edges:
+        Reducer 2 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE), Union 3 
(CONTAINS)
+        Reducer 4 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
+        Reducer 5 <- Reducer 4 (SIMPLE_EDGE), Reducer 9 (SIMPLE_EDGE), Union 3 
(CONTAINS)
+        Reducer 6 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
+        Reducer 7 <- Reducer 6 (SIMPLE_EDGE), Union 3 (CONTAINS)
+        Reducer 8 <- Map 1 (SIMPLE_EDGE), Map 10 (SIMPLE_EDGE)
+        Reducer 9 <- Reducer 8 (SIMPLE_EDGE)
+#### A masked pattern was here ####
+      Vertices:
+        Map 1 
+            Map Operator Tree:
+                TableScan
+                  alias: target_ice
+                  Statistics: Num rows: 4 Data size: 16 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: PARTITION__SPEC__ID (type: int), 
PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: 
bigint), a (type: int)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4
+                    Statistics: Num rows: 4 Data size: 832 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col4 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col4 (type: int)
+                      Statistics: Num rows: 4 Data size: 832 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: 
bigint), _col2 (type: string), _col3 (type: bigint)
+                  Filter Operator
+                    predicate: FILE__PATH is not null (type: boolean)
+                    Statistics: Num rows: 4 Data size: 380 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: PARTITION__SPEC__ID (type: int), 
PARTITION__HASH (type: bigint), FILE__PATH (type: string), ROW__POSITION (type: 
bigint), a (type: int), b (type: string), c (type: int)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6
+                      Statistics: Num rows: 4 Data size: 1196 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col4 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col4 (type: int)
+                        Statistics: Num rows: 4 Data size: 1196 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: 
bigint), _col2 (type: string), _col3 (type: bigint), _col5 (type: string), 
_col6 (type: int)
+                  Filter Operator
+                    predicate: (a > 100) (type: boolean)
+                    Statistics: Num rows: 3 Data size: 285 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: PARTITION__SPEC__ID (type: int), 
PARTITION__HASH (type: bigint), FILE__PATH (type: string), a (type: int), b 
(type: string), c (type: int)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5
+                      Statistics: Num rows: 3 Data size: 873 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col3 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col3 (type: int)
+                        Statistics: Num rows: 3 Data size: 873 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: int), _col1 (type: 
bigint), _col2 (type: string), _col4 (type: string), _col5 (type: int)
+                  Filter Operator
+                    predicate: ((a > 100) and FILE__PATH is not null) (type: 
boolean)
+                    Statistics: Num rows: 3 Data size: 12 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: FILE__PATH (type: string), a (type: int)
+                      outputColumnNames: _col0, _col1
+                      Statistics: Num rows: 3 Data size: 564 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col1 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col1 (type: int)
+                        Statistics: Num rows: 3 Data size: 564 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        value expressions: _col0 (type: string)
+            Execution mode: vectorized
+        Map 10 
+            Map Operator Tree:
+                TableScan
+                  alias: src
+                  Statistics: Num rows: 6 Data size: 576 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Select Operator
+                    expressions: a (type: int), b (type: string), c (type: int)
+                    outputColumnNames: _col0, _col1, _col2
+                    Statistics: Num rows: 6 Data size: 576 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col0 (type: int)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col0 (type: int)
+                      Statistics: Num rows: 6 Data size: 576 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col1 (type: string), _col2 (type: 
int)
+                  Filter Operator
+                    predicate: (a > 100) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 4 Basic stats: COMPLETE 
Column stats: COMPLETE
+                    Select Operator
+                      expressions: a (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 4 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 1 Data size: 4 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Filter Operator
+                    predicate: a is not null (type: boolean)
+                    Statistics: Num rows: 6 Data size: 24 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: a (type: int)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 6 Data size: 24 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Reduce Output Operator
+                        key expressions: _col0 (type: int)
+                        null sort order: z
+                        sort order: +
+                        Map-reduce partition columns: _col0 (type: int)
+                        Statistics: Num rows: 6 Data size: 24 Basic stats: 
COMPLETE Column stats: COMPLETE
+            Execution mode: vectorized
+        Reducer 2 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Full Outer Join 0 to 1
+                keys:
+                  0 _col4 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, 
_col6, _col7
+                Statistics: Num rows: 14 Data size: 1520 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Filter Operator
+                  predicate: _col4 is null (type: boolean)
+                  Statistics: Num rows: 10 Data size: 1216 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: int), _col1 (type: bigint), 
_col2 (type: string), _col3 (type: bigint), _col5 (type: int), _col6 (type: 
string), _col7 (type: int)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6
+                    Statistics: Num rows: 10 Data size: 1200 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    File Output Operator
+                      compressed: false
+                      Statistics: Num rows: 13 Data size: 2097 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      table:
+                          input format: 
org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+                          output format: 
org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+                          serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+                          name: default.target_ice
+        Reducer 4 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Outer Join 0 to 1
+                keys:
+                  0 _col4 (type: int)
+                  1 _col0 (type: int)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, 
_col6, _col7
+                Statistics: Num rows: 8 Data size: 2412 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Filter Operator
+                  predicate: ((((_col4 <> _col7) or (_col4 <= 100)) and _col4 
is not null) or (((_col4 = _col7) and (_col4 > 100)) or _col4 is null) is null) 
(type: boolean)
+                  Statistics: Num rows: 8 Data size: 2412 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  Select Operator
+                    expressions: _col0 (type: int), _col1 (type: bigint), 
_col2 (type: string), _col3 (type: bigint), _col4 (type: int), _col5 (type: 
string), _col6 (type: int)
+                    outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6
+                    Statistics: Num rows: 8 Data size: 2392 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Reduce Output Operator
+                      key expressions: _col2 (type: string)
+                      null sort order: z
+                      sort order: +
+                      Map-reduce partition columns: _col2 (type: string)
+                      Statistics: Num rows: 8 Data size: 2392 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      value expressions: _col0 (type: int), _col1 (type: 
bigint), _col3 (type: bigint), _col4 (type: int), _col5 (type: string), _col6 
(type: int)
+        Reducer 5 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Left Semi Join 0 to 1
+                keys:
+                  0 _col2 (type: string)
+                  1 _col0 (type: string)
+                outputColumnNames: _col0, _col1, _col2, _col3, _col4, _col5, 
_col6
+                Statistics: Num rows: 2 Data size: 598 Basic stats: COMPLETE 
Column stats: COMPLETE
+                File Output Operator
+                  compressed: false
+                  Statistics: Num rows: 13 Data size: 2097 Basic stats: 
COMPLETE Column stats: COMPLETE
+                  table:
+                      input format: 
org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+                      output format: 
org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+                      serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+                      name: default.target_ice
+        Reducer 6 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col3 (type: int)
+                outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6
+                Statistics: Num rows: 1 Data size: 291 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col3 (type: string)
+                  null sort order: a
+                  sort order: +
+                  Map-reduce partition columns: _col3 (type: string)
+                  Statistics: Num rows: 1 Data size: 291 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  value expressions: _col1 (type: int), _col2 (type: bigint), 
_col4 (type: int), _col5 (type: string), _col6 (type: int)
+        Reducer 7 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Select Operator
+                expressions: VALUE._col1 (type: int), VALUE._col2 (type: 
bigint), KEY.reducesinkkey0 (type: string), VALUE._col3 (type: int), 
VALUE._col4 (type: string), VALUE._col5 (type: int)
+                outputColumnNames: _col1, _col2, _col3, _col4, _col5, _col6
+                Statistics: Num rows: 1 Data size: 291 Basic stats: COMPLETE 
Column stats: COMPLETE
+                PTF Operator
+                  Function definitions:
+                      Input definition
+                        input alias: ptf_0
+                        type: WINDOWING
+                      Windowing table definition
+                        input alias: ptf_1
+                        name: windowingtablefunction
+                        order by: _col3 ASC NULLS FIRST
+                        partition by: _col3
+                        raw input shape:
+                        window functions:
+                            window function definition
+                              alias: row_number_window_0
+                              name: row_number
+                              window function: GenericUDAFRowNumberEvaluator
+                              window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
+                              isPivotResult: true
+                  Statistics: Num rows: 1 Data size: 291 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (row_number_window_0 = 1) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 291 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: _col1 (type: int), _col2 (type: bigint), 
_col3 (type: string), -1L (type: bigint), _col4 (type: int), _col5 (type: 
string), _col6 (type: int)
+                      outputColumnNames: _col0, _col1, _col2, _col3, _col4, 
_col5, _col6
+                      Statistics: Num rows: 1 Data size: 299 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      File Output Operator
+                        compressed: false
+                        Statistics: Num rows: 13 Data size: 2097 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        table:
+                            input format: 
org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+                            output format: 
org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+                            serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+                            name: default.target_ice
+        Reducer 8 
+            Reduce Operator Tree:
+              Merge Join Operator
+                condition map:
+                     Inner Join 0 to 1
+                keys:
+                  0 _col0 (type: int)
+                  1 _col1 (type: int)
+                outputColumnNames: _col1
+                Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE 
Column stats: COMPLETE
+                Reduce Output Operator
+                  key expressions: _col1 (type: string)
+                  null sort order: a
+                  sort order: +
+                  Map-reduce partition columns: _col1 (type: string)
+                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE 
Column stats: COMPLETE
+        Reducer 9 
+            Execution mode: vectorized
+            Reduce Operator Tree:
+              Select Operator
+                expressions: KEY.reducesinkkey0 (type: string)
+                outputColumnNames: _col1
+                Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE 
Column stats: COMPLETE
+                PTF Operator
+                  Function definitions:
+                      Input definition
+                        input alias: ptf_0
+                        output shape: _col1: string
+                        type: WINDOWING
+                      Windowing table definition
+                        input alias: ptf_1
+                        name: windowingtablefunction
+                        order by: _col1 ASC NULLS FIRST
+                        partition by: _col1
+                        raw input shape:
+                        window functions:
+                            window function definition
+                              alias: row_number_window_0
+                              name: row_number
+                              window function: GenericUDAFRowNumberEvaluator
+                              window frame: ROWS PRECEDING(MAX)~FOLLOWING(MAX)
+                              isPivotResult: true
+                  Statistics: Num rows: 1 Data size: 184 Basic stats: COMPLETE 
Column stats: COMPLETE
+                  Filter Operator
+                    predicate: (row_number_window_0 = 1) (type: boolean)
+                    Statistics: Num rows: 1 Data size: 184 Basic stats: 
COMPLETE Column stats: COMPLETE
+                    Select Operator
+                      expressions: _col1 (type: string)
+                      outputColumnNames: _col0
+                      Statistics: Num rows: 1 Data size: 184 Basic stats: 
COMPLETE Column stats: COMPLETE
+                      Group By Operator
+                        keys: _col0 (type: string)
+                        minReductionHashAggr: 0.4
+                        mode: hash
+                        outputColumnNames: _col0
+                        Statistics: Num rows: 1 Data size: 184 Basic stats: 
COMPLETE Column stats: COMPLETE
+                        Reduce Output Operator
+                          key expressions: _col0 (type: string)
+                          null sort order: z
+                          sort order: +
+                          Map-reduce partition columns: _col0 (type: string)
+                          Statistics: Num rows: 1 Data size: 184 Basic stats: 
COMPLETE Column stats: COMPLETE
+        Union 3 
+            Vertex: Union 3
+
+  Stage: Stage-2
+    Dependency Collection
+
+  Stage: Stage-0
+    Move Operator
+      tables:
+          replace: false
+          table:
+              input format: org.apache.iceberg.mr.hive.HiveIcebergInputFormat
+              output format: org.apache.iceberg.mr.hive.HiveIcebergOutputFormat
+              serde: org.apache.iceberg.mr.hive.HiveIcebergSerDe
+              name: default.target_ice
+
+  Stage: Stage-3
+    Stats Work
+      Basic Stats Work:
+
 PREHOOK: query: explain
 merge into target_ice as t using source src ON t.a = src.a
 when matched and t.a > 100 THEN DELETE
diff --git 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java
 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java
index f422d5514bb..968f34078af 100644
--- 
a/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java
+++ 
b/ql/src/java/org/apache/hadoop/hive/ql/parse/rewrite/CopyOnWriteMergeRewriter.java
@@ -127,6 +127,7 @@ public class CopyOnWriteMergeRewriter extends MergeRewriter 
{
   static class CopyOnWriteMergeWhenClauseSqlGenerator extends 
MergeRewriter.MergeWhenClauseSqlGenerator {
 
     private final COWWithClauseBuilder cowWithClauseBuilder;
+    private int subQueryCount = 0;
 
     CopyOnWriteMergeWhenClauseSqlGenerator(
       HiveConf conf, MultiInsertSqlGenerator sqlGenerator, MergeStatement 
mergeStatement) {
@@ -138,7 +139,7 @@ public class CopyOnWriteMergeRewriter extends MergeRewriter 
{
     public void appendWhenNotMatchedInsertClause(MergeStatement.InsertClause 
insertClause) {
       String targetAlias = mergeStatement.getTargetAlias();
       
-      if (mergeStatement.getWhenClauses().size() > 1) {
+      if (++subQueryCount > 1) {
         sqlGenerator.append("union all\n");
       }
       sqlGenerator.append("    -- insert clause\n").append("SELECT ");
@@ -173,7 +174,8 @@ public class CopyOnWriteMergeRewriter extends MergeRewriter 
{
 
       UnaryOperator<String> columnRefsFunc = value -> 
replaceColumnRefsWithTargetPrefix(targetAlias, value);
       sqlGenerator.append("    -- update clause (insert 
part)\n").append("SELECT ");
-
+      ++subQueryCount;
+      
       if (isNotBlank(hintStr)) {
         sqlGenerator.append(hintStr);
         hintStr = null;
@@ -206,8 +208,7 @@ public class CopyOnWriteMergeRewriter extends MergeRewriter 
{
       UnaryOperator<String> columnRefsFunc = value -> 
replaceColumnRefsWithTargetPrefix(targetAlias, value);
       List<String> deleteValues = 
sqlGenerator.getDeleteValues(Context.Operation.DELETE);
 
-      List<MergeStatement.WhenClause> whenClauses = 
mergeStatement.getWhenClauses();
-      if (whenClauses.size() > 1 || whenClauses.get(0) instanceof 
MergeStatement.UpdateClause) {
+      if (++subQueryCount > 1) {
         sqlGenerator.append("union all\n");
       }
       sqlGenerator.append("    -- delete clause\n").append("SELECT ");

Reply via email to