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 293a448296 HIVE-26535: Iceberg: Support adding parquet compression
type via Table properties. (#3597). (Ayush Saxena, reviewed by Adam Szita)
293a448296 is described below
commit 293a448296933b7498a91e7eeb91edc88dfaa07e
Author: Ayush Saxena <[email protected]>
AuthorDate: Thu Sep 15 19:45:35 2022 +0530
HIVE-26535: Iceberg: Support adding parquet compression type via Table
properties. (#3597). (Ayush Saxena, reviewed by Adam Szita)
---
.../org/apache/iceberg/hive/HiveTableOperations.java | 5 +++--
.../mr/hive/TestHiveIcebergStorageHandlerNoScan.java | 19 +++++++++++++++++++
2 files changed, 22 insertions(+), 2 deletions(-)
diff --git
a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
index 6afcd71408..8fdcb6b2e5 100644
---
a/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
+++
b/iceberg/iceberg-catalog/src/main/java/org/apache/iceberg/hive/HiveTableOperations.java
@@ -70,6 +70,7 @@ import
org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap;
import org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.util.Tasks;
+import org.apache.parquet.hadoop.ParquetOutputFormat;
import org.apache.thrift.TException;
import org.slf4j.Logger;
import org.slf4j.LoggerFactory;
@@ -96,8 +97,8 @@ public class HiveTableOperations extends
BaseMetastoreTableOperations {
private static final BiMap<String, String> ICEBERG_TO_HMS_TRANSLATION =
ImmutableBiMap.of(
// gc.enabled in Iceberg and external.table.purge in Hive are meant to
do the same things but with different names
- GC_ENABLED, "external.table.purge"
- );
+ GC_ENABLED, "external.table.purge",
+ TableProperties.PARQUET_COMPRESSION, ParquetOutputFormat.COMPRESSION);
private static Cache<String, ReentrantLock> commitLockCache;
diff --git
a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
index dd76545aac..7890d32e91 100644
---
a/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
+++
b/iceberg/iceberg-handler/src/test/java/org/apache/iceberg/mr/hive/TestHiveIcebergStorageHandlerNoScan.java
@@ -72,6 +72,7 @@ import
org.apache.iceberg.relocated.com.google.common.collect.Lists;
import org.apache.iceberg.relocated.com.google.common.collect.Maps;
import org.apache.iceberg.types.Type;
import org.apache.iceberg.types.Types;
+import org.apache.parquet.hadoop.ParquetOutputFormat;
import org.apache.thrift.TException;
import org.junit.After;
import org.junit.AfterClass;
@@ -1623,6 +1624,24 @@ public class TestHiveIcebergStorageHandlerNoScan {
});
}
+ @Test
+ public void testParquetHiveCatalogValidation() throws TException,
InterruptedException, IOException {
+
+ // Create a table with explicitly set parquet.compression
+ TableIdentifier target = TableIdentifier.of("default", "target");
+ Table table = testTables.createTable(shell, target.name(),
HiveIcebergStorageHandlerTestUtils.CUSTOMER_SCHEMA,
+ PartitionSpec.unpartitioned(), FileFormat.PARQUET,
HiveIcebergStorageHandlerTestUtils.CUSTOMER_RECORDS, 1,
+ Collections.singletonMap(ParquetOutputFormat.COMPRESSION, "SNAPPY"));
+
+ // Check the property got set in the hive table metadata.
+ org.apache.hadoop.hive.metastore.api.Table hmsTable =
shell.metastore().getTable(target);
+ Assert.assertEquals("SNAPPY",
hmsTable.getParameters().get(ParquetOutputFormat.COMPRESSION).toUpperCase());
+
+ // Check the property got set in the iceberg table metadata.
+ Table icebergTable = testTables.loadTable(target);
+ Assert.assertEquals("SNAPPY",
icebergTable.properties().get(TableProperties.PARQUET_COMPRESSION).toUpperCase());
+ }
+
/**
* Checks that the new schema has newintcol and newstring col columns on
both HMS and Iceberg sides