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 8d017fbbfe1 HIVE-29218: LOAD OVERWRITE PARTITION on muti-level 
partititoned external Iceberg table may unexpectedly delete other partitions 
(#6090)
8d017fbbfe1 is described below

commit 8d017fbbfe1deee50408e1c01a3d5d4c0dc7d4e8
Author: jaredjia <[email protected]>
AuthorDate: Fri Sep 26 16:11:13 2025 +0800

    HIVE-29218: LOAD OVERWRITE PARTITION on muti-level partititoned external 
Iceberg table may unexpectedly delete other partitions (#6090)
---
 .../pcol1=x/pcol2=y/000000_0.parquet               | Bin 0 -> 1153 bytes
 .../pcol1=x/pcol2=z/000000_0.parquet               | Bin 0 -> 1153 bytes
 .../org/apache/iceberg/mr/hive/HiveTableUtil.java  |  11 ++---
 .../apache/iceberg/mr/hive/IcebergTableUtil.java   |  16 +++----
 .../src/test/queries/positive/iceberg_load_data.q  |  18 +++++++-
 .../test/results/positive/iceberg_load_data.q.out  |  50 +++++++++++++++++++++
 6 files changed, 77 insertions(+), 18 deletions(-)

diff --git 
a/data/files/parquet_multi_partition/pcol1=x/pcol2=y/000000_0.parquet 
b/data/files/parquet_multi_partition/pcol1=x/pcol2=y/000000_0.parquet
new file mode 100644
index 00000000000..2446f368fc5
Binary files /dev/null and 
b/data/files/parquet_multi_partition/pcol1=x/pcol2=y/000000_0.parquet differ
diff --git 
a/data/files/parquet_multi_partition/pcol1=x/pcol2=z/000000_0.parquet 
b/data/files/parquet_multi_partition/pcol1=x/pcol2=z/000000_0.parquet
new file mode 100644
index 00000000000..6bb7c2df820
Binary files /dev/null and 
b/data/files/parquet_multi_partition/pcol1=x/pcol2=z/000000_0.parquet differ
diff --git 
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java
 
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java
index daf0d1e525e..446a02154f8 100644
--- 
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java
+++ 
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveTableUtil.java
@@ -59,6 +59,7 @@
 import org.apache.iceberg.Transaction;
 import org.apache.iceberg.data.TableMigrationUtil;
 import org.apache.iceberg.exceptions.NotFoundException;
+import org.apache.iceberg.expressions.Expression;
 import org.apache.iceberg.expressions.Expressions;
 import org.apache.iceberg.hadoop.HadoopConfigurable;
 import org.apache.iceberg.hadoop.HadoopFileIO;
@@ -71,8 +72,6 @@
 import org.apache.iceberg.mr.InputFormatConfig;
 import org.apache.iceberg.relocated.com.google.common.collect.Lists;
 import 
org.apache.iceberg.relocated.com.google.common.util.concurrent.ThreadFactoryBuilder;
-import org.apache.iceberg.types.Conversions;
-import org.apache.iceberg.types.Type;
 import org.apache.iceberg.util.SerializationUtil;
 import org.slf4j.Logger;
 import org.slf4j.LoggerFactory;
@@ -171,11 +170,9 @@ public static void appendFiles(URI fromURI, String format, 
Table icebergTbl, boo
       if (isOverwrite) {
         DeleteFiles delete = transaction.newDelete();
         if (partitionSpec != null) {
-          for (Map.Entry<String, String> part : partitionSpec.entrySet()) {
-            final Type partKeyType = 
icebergTbl.schema().findType(part.getKey());
-            final Object partKeyVal = 
Conversions.fromPartitionString(partKeyType, part.getValue());
-            delete.deleteFromRowFilter(Expressions.equal(part.getKey(), 
partKeyVal));
-          }
+          Expression partitionExpr =
+              IcebergTableUtil.generateExpressionFromPartitionSpec(icebergTbl, 
partitionSpec, true);
+          delete.deleteFromRowFilter(partitionExpr);
         } else {
           delete.deleteFromRowFilter(Expressions.alwaysTrue());
         }
diff --git 
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java
 
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java
index ea6e789cbfe..0ce4e255c2b 100644
--- 
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java
+++ 
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/IcebergTableUtil.java
@@ -23,7 +23,6 @@
 import java.nio.ByteBuffer;
 import java.time.ZoneId;
 import java.util.Collection;
-import java.util.Collections;
 import java.util.Comparator;
 import java.util.Iterator;
 import java.util.List;
@@ -491,20 +490,17 @@ public static Expression 
generateExpressionFromPartitionSpec(Table table, Map<St
       String partColName = entry.getKey();
       if (partitionFieldMap.containsKey(partColName)) {
         PartitionField partitionField = partitionFieldMap.get(partColName);
-        Type resultType = 
partitionField.transform().getResultType(table.schema()
-            .findField(partitionField.sourceId()).type());
-        Object value = Conversions.fromPartitionString(resultType, 
entry.getValue());
-        TransformSpec.TransformType transformType = 
TransformSpec.fromString(partitionField.transform().toString());
-        Iterable<?> iterable = () -> 
Collections.singletonList(value).iterator();
-        if (TransformSpec.TransformType.IDENTITY == transformType) {
-          Expression boundPredicate = Expressions.in(partitionField.name(), 
iterable);
+        if (partitionField.transform().isIdentity()) {
+          final Type partKeyType = 
table.schema().findField(partitionField.sourceId()).type();
+          final Object partKeyVal = 
Conversions.fromPartitionString(partKeyType, entry.getValue());
+          Expression boundPredicate = Expressions.equal(partColName, 
partKeyVal);
           finalExp = Expressions.and(finalExp, boundPredicate);
         } else {
           throw new SemanticException(
-              String.format("Partition transforms are not supported via 
truncate operation: %s", partColName));
+              String.format("Partition transforms are not supported here: %s", 
partColName));
         }
       } else {
-        throw new SemanticException(String.format("No partition 
column/transform by the name: %s", partColName));
+        throw new SemanticException(String.format("No partition column by the 
name: %s", partColName));
       }
     }
     return finalExp;
diff --git 
a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_load_data.q 
b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_load_data.q
index 23c32bd2490..a23b905b429 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/iceberg_load_data.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/iceberg_load_data.q
@@ -83,4 +83,20 @@ explain LOAD DATA LOCAL INPATH 
'../../data/files/parquet_partition/pcol=200' OVE
 LOAD DATA LOCAL INPATH '../../data/files/parquet_partition/pcol=200' OVERWRITE 
INTO TABLE
 ice_parquet_partitioned PARTITION (pcol='200');
 
-select * from ice_parquet_partitioned order by intcol;
\ No newline at end of file
+select * from ice_parquet_partitioned order by intcol;
+
+create external table ice_parquet_multi_partitioned (
+    strcol string,
+    intcol integer
+) partitioned by (pcol1 string, pcol2 string)
+stored by iceberg;
+
+LOAD DATA LOCAL INPATH 
'../../data/files/parquet_multi_partition/pcol1=x/pcol2=y' INTO TABLE
+ice_parquet_multi_partitioned PARTITION (pcol1='x', pcol2='y');
+
+select * from ice_parquet_multi_partitioned order by pcol1, pcol2;
+
+LOAD DATA LOCAL INPATH 
'../../data/files/parquet_multi_partition/pcol1=x/pcol2=z' OVERWRITE INTO TABLE
+ice_parquet_multi_partitioned PARTITION (pcol1='x', pcol2='z');
+
+select * from ice_parquet_multi_partitioned order by pcol1, pcol2;
\ No newline at end of file
diff --git 
a/iceberg/iceberg-handler/src/test/results/positive/iceberg_load_data.q.out 
b/iceberg/iceberg-handler/src/test/results/positive/iceberg_load_data.q.out
index 3e2da4fe5f1..a364be2dedf 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/iceberg_load_data.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/iceberg_load_data.q.out
@@ -502,3 +502,53 @@ c  3       200
 d      4       200
 AA     10      100
 CC     30      300
+PREHOOK: query: create external table ice_parquet_multi_partitioned (
+    strcol string,
+    intcol integer
+) partitioned by (pcol1 string, pcol2 string)
+stored by iceberg
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@ice_parquet_multi_partitioned
+POSTHOOK: query: create external table ice_parquet_multi_partitioned (
+    strcol string,
+    intcol integer
+) partitioned by (pcol1 string, pcol2 string)
+stored by iceberg
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@ice_parquet_multi_partitioned
+PREHOOK: query: LOAD DATA LOCAL INPATH 
'../../data/files/parquet_multi_partition/pcol1=x/pcol2=y' INTO TABLE
+ice_parquet_multi_partitioned PARTITION (pcol1='x', pcol2='y')
+PREHOOK: type: LOAD
+POSTHOOK: query: LOAD DATA LOCAL INPATH 
'../../data/files/parquet_multi_partition/pcol1=x/pcol2=y' INTO TABLE
+ice_parquet_multi_partitioned PARTITION (pcol1='x', pcol2='y')
+POSTHOOK: type: LOAD
+PREHOOK: query: select * from ice_parquet_multi_partitioned order by pcol1, 
pcol2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice_parquet_multi_partitioned
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from ice_parquet_multi_partitioned order by pcol1, 
pcol2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice_parquet_multi_partitioned
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+a      1       x       y
+b      2       x       y
+PREHOOK: query: LOAD DATA LOCAL INPATH 
'../../data/files/parquet_multi_partition/pcol1=x/pcol2=z' OVERWRITE INTO TABLE
+ice_parquet_multi_partitioned PARTITION (pcol1='x', pcol2='z')
+PREHOOK: type: LOAD
+POSTHOOK: query: LOAD DATA LOCAL INPATH 
'../../data/files/parquet_multi_partition/pcol1=x/pcol2=z' OVERWRITE INTO TABLE
+ice_parquet_multi_partitioned PARTITION (pcol1='x', pcol2='z')
+POSTHOOK: type: LOAD
+PREHOOK: query: select * from ice_parquet_multi_partitioned order by pcol1, 
pcol2
+PREHOOK: type: QUERY
+PREHOOK: Input: default@ice_parquet_multi_partitioned
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from ice_parquet_multi_partitioned order by pcol1, 
pcol2
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@ice_parquet_multi_partitioned
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+a      1       x       y
+b      2       x       y
+A      10      x       z
+B      20      x       z

Reply via email to