Skip to content

Commit

Permalink
HIVE-28764: Iceberg: Throw Exception in case of Drop Partition on tra…
Browse files Browse the repository at this point in the history
…nsformed column. (#5644). (Ayush Saxena, reviewed by Simhadri Govindappa)
  • Loading branch information
ayushtkn authored Feb 15, 2025
1 parent b395fc7 commit 86530ca
Show file tree
Hide file tree
Showing 3 changed files with 85 additions and 0 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand Down Expand Up @@ -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
Expand Down
Original file line number Diff line number Diff line change
@@ -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;
Original file line number Diff line number Diff line change
@@ -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

0 comments on commit 86530ca

Please sign in to comment.