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

ayushsaxena 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 86530ca1568 HIVE-28764: Iceberg: Throw Exception in case of Drop 
Partition on transformed column. (#5644). (Ayush Saxena, reviewed by Simhadri 
Govindappa)
86530ca1568 is described below

commit 86530ca15684b116f81c63d230515194c32b06bf
Author: Ayush Saxena <ayushsax...@apache.org>
AuthorDate: Sun Feb 16 02:25:03 2025 +0530

    HIVE-28764: Iceberg: Throw Exception in case of Drop Partition on 
transformed column. (#5644). (Ayush Saxena, reviewed by Simhadri Govindappa)
---
 .../iceberg/mr/hive/HiveIcebergMetaHook.java       |  8 ++++
 .../iceberg_drop_partition_tranform_column.q       | 24 ++++++++++
 .../iceberg_drop_partition_tranform_column.q.out   | 53 ++++++++++++++++++++++
 3 files changed, 85 insertions(+)

diff --git 
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
 
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
index b2a8080e56a..9d41e636dfe 100644
--- 
a/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
+++ 
b/iceberg/iceberg-handler/src/main/java/org/apache/iceberg/mr/hive/HiveIcebergMetaHook.java
@@ -60,6 +60,7 @@
 import org.apache.hadoop.hive.ql.exec.SerializationUtilities;
 import org.apache.hadoop.hive.ql.io.AcidUtils;
 import org.apache.hadoop.hive.ql.io.sarg.ConvertAstToSearchArg;
+import org.apache.hadoop.hive.ql.io.sarg.PredicateLeaf;
 import org.apache.hadoop.hive.ql.io.sarg.SearchArgument;
 import org.apache.hadoop.hive.ql.metadata.Hive;
 import org.apache.hadoop.hive.ql.metadata.HiveException;
@@ -1129,11 +1130,18 @@ public void 
preDropPartitions(org.apache.hadoop.hive.metastore.api.Table hmsTabl
       List<org.apache.commons.lang3.tuple.Pair<Integer, byte[]>> partExprs)
       throws MetaException {
     Table icebergTbl = IcebergTableUtil.getTable(conf, hmsTable);
+    Map<String, PartitionField> partitionFieldMap =
+        
icebergTbl.spec().fields().stream().collect(Collectors.toMap(PartitionField::name,
 Function.identity()));
     DeleteFiles deleteFiles = icebergTbl.newDelete();
     List<Expression> expressions = partExprs.stream().map(partExpr -> {
       ExprNodeDesc exprNodeDesc = SerializationUtilities
           .deserializeObjectWithTypeInformation(partExpr.getRight(), true);
       SearchArgument sarg = ConvertAstToSearchArg.create(conf, 
(ExprNodeGenericFuncDesc) exprNodeDesc);
+      for (PredicateLeaf leaf : sarg.getLeaves()) {
+        if (leaf.getColumnName() != null && 
!partitionFieldMap.containsKey(leaf.getColumnName())) {
+          throw new UnsupportedOperationException("Drop Partition not 
supported on Transformed Columns");
+        }
+      }
       return HiveIcebergFilterFactory.generateFilterExpression(sarg);
     }).collect(Collectors.toList());
     PartitionsTable partitionsTable = (PartitionsTable) MetadataTableUtils
diff --git 
a/iceberg/iceberg-handler/src/test/queries/negative/iceberg_drop_partition_tranform_column.q
 
b/iceberg/iceberg-handler/src/test/queries/negative/iceberg_drop_partition_tranform_column.q
new file mode 100644
index 00000000000..15d98ada7ac
--- /dev/null
+++ 
b/iceberg/iceberg-handler/src/test/queries/negative/iceberg_drop_partition_tranform_column.q
@@ -0,0 +1,24 @@
+CREATE TABLE drop_partition (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, days(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+);
+
+INSERT INTO drop_partition VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0);
+
+
+ALTER TABLE drop_partition DROP PARTITION (location_id = 'loc_002', 
reading_time = '2024-06-01 10:15:00');
+
+SELECT * FROM drop_partition;
\ No newline at end of file
diff --git 
a/iceberg/iceberg-handler/src/test/results/negative/iceberg_drop_partition_tranform_column.q.out
 
b/iceberg/iceberg-handler/src/test/results/negative/iceberg_drop_partition_tranform_column.q.out
new file mode 100644
index 00000000000..dee16170c04
--- /dev/null
+++ 
b/iceberg/iceberg-handler/src/test/results/negative/iceberg_drop_partition_tranform_column.q.out
@@ -0,0 +1,53 @@
+PREHOOK: query: CREATE TABLE drop_partition (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, days(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@drop_partition
+POSTHOOK: query: CREATE TABLE drop_partition (
+sensor_id STRING,
+location_id STRING,
+reading_time TIMESTAMP,
+temperature DOUBLE,
+humidity DOUBLE
+)
+PARTITIONED BY SPEC (location_id, days(reading_time))
+STORED BY ICEBERG
+TBLPROPERTIES (
+'write.format.default'='parquet',
+'format-version'='2',
+'write.parquet.compression-codec'='gzip'
+)
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@drop_partition
+PREHOOK: query: INSERT INTO drop_partition VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0)
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@drop_partition
+POSTHOOK: query: INSERT INTO drop_partition VALUES
+('sensor_001', 'loc_001', '2024-06-01 10:00:00', 22.5, 60.0),
+('sensor_002', 'loc_002', '2024-06-01 10:15:00', 23.0, 58.0),
+('sensor_001', 'loc_001', '2024-06-02 11:00:00', 22.8, 61.0)
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@drop_partition
+PREHOOK: query: ALTER TABLE drop_partition DROP PARTITION (location_id = 
'loc_002', reading_time = '2024-06-01 10:15:00')
+PREHOOK: type: ALTERTABLE_DROPPARTS
+PREHOOK: Input: default@drop_partition
+PREHOOK: Output: 
default@drop_partition@location_id=loc_002/reading_time_day=2024-06-01
+FAILED: Execution Error, return code 40000 from 
org.apache.hadoop.hive.ql.ddl.DDLTask. Drop Partition not supported on 
Transformed Columns

Reply via email to