You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@hive.apache.org by dk...@apache.org on 2023/05/24 07:29:35 UTC

[hive] branch master updated: HIVE-27367: Iceberg: Encode partition key values to avoid unnecessary partition keys split in DataFiles.fillFromPath (Denys Kuzmenko, reviewed by Attila Turoczy, Ayush Saxena)

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 a0845893c9b HIVE-27367: Iceberg: Encode partition key values to avoid unnecessary partition keys split in DataFiles.fillFromPath (Denys Kuzmenko, reviewed by Attila Turoczy, Ayush Saxena)
a0845893c9b is described below

commit a0845893c9b3ee53f099a1be47dd487d6b4d6bdd
Author: Denys Kuzmenko <de...@gmail.com>
AuthorDate: Wed May 24 10:29:22 2023 +0300

    HIVE-27367: Iceberg: Encode partition key values to avoid unnecessary partition keys split in DataFiles.fillFromPath (Denys Kuzmenko, reviewed by Attila Turoczy, Ayush Saxena)
    
    Closes #4339
---
 .../org/apache/iceberg/mr/hive/HiveTableUtil.java  | 20 +++++++-
 .../test/queries/positive/alter_table_to_iceberg.q | 14 +++++-
 .../results/positive/alter_table_to_iceberg.q.out  | 57 ++++++++++++++++++++++
 3 files changed, 88 insertions(+), 3 deletions(-)

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 fd45ff488f1..a02392417bf 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
@@ -30,6 +30,7 @@ import java.util.Properties;
 import java.util.concurrent.Callable;
 import java.util.concurrent.ExecutorService;
 import java.util.concurrent.Executors;
+import java.util.stream.IntStream;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.LocatedFileStatus;
@@ -42,11 +43,13 @@ import org.apache.hadoop.hive.metastore.api.MetaException;
 import org.apache.hadoop.hive.metastore.api.Partition;
 import org.apache.hadoop.hive.metastore.api.hive_metastoreConstants;
 import org.apache.hadoop.hive.metastore.partition.spec.PartitionSpecProxy;
+import org.apache.hadoop.hive.metastore.utils.FileUtils;
 import org.apache.iceberg.AppendFiles;
 import org.apache.iceberg.DataFile;
 import org.apache.iceberg.MetricsConfig;
 import org.apache.iceberg.PartitionSpec;
 import org.apache.iceberg.SerializableTable;
+import org.apache.iceberg.StructLike;
 import org.apache.iceberg.Table;
 import org.apache.iceberg.TableProperties;
 import org.apache.iceberg.data.TableMigrationUtil;
@@ -146,8 +149,21 @@ public class HiveTableUtil {
       if (fileName.startsWith(".") || fileName.startsWith("_") || fileName.endsWith("metadata.json")) {
         continue;
       }
-      dataFiles.addAll(TableMigrationUtil.listPartition(partitionKeys, fileStatus.getPath().toString(), format, spec,
-          conf, metricsConfig, nameMapping));
+      partitionKeys.replaceAll((key, value) -> FileUtils.escapePathName(value));
+
+      int[] stringFields = IntStream.range(0, spec.javaClasses().length)
+        .filter(i -> spec.javaClasses()[i].isAssignableFrom(String.class)).toArray();
+
+      dataFiles.addAll(Lists.transform(
+          TableMigrationUtil.listPartition(partitionKeys, fileStatus.getPath().toString(), format, spec,
+            conf, metricsConfig, nameMapping),
+          dataFile -> {
+            StructLike structLike = dataFile.partition();
+            for (int pos : stringFields) {
+              structLike.set(pos, FileUtils.unescapePathName(structLike.get(pos, String.class)));
+            }
+            return dataFile;
+          }));
     }
     return dataFiles;
   }
diff --git a/iceberg/iceberg-handler/src/test/queries/positive/alter_table_to_iceberg.q b/iceberg/iceberg-handler/src/test/queries/positive/alter_table_to_iceberg.q
index a9ed7d48808..6564515d25d 100644
--- a/iceberg/iceberg-handler/src/test/queries/positive/alter_table_to_iceberg.q
+++ b/iceberg/iceberg-handler/src/test/queries/positive/alter_table_to_iceberg.q
@@ -44,4 +44,16 @@ explain alter table tbl_avro convert to iceberg;
 alter table tbl_avro convert to iceberg;
 describe formatted tbl_avro;
 select * from tbl_avro order by a;
-drop table tbl_avro;
\ No newline at end of file
+drop table tbl_avro;
+
+
+set hive.exec.dynamic.partition.mode=nonstrict;
+
+drop table if exists part_tbl_parquet;
+create external table part_tbl_parquet (a int) partitioned by (s string) stored as parquet;
+insert into part_tbl_parquet partition (s) values (1, '2023/05/18');
+select * from part_tbl_parquet;
+
+alter table part_tbl_parquet convert to iceberg;
+select * from part_tbl_parquet;
+drop table part_tbl_parquet;
\ No newline at end of file
diff --git a/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out b/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out
index f6c92cc8824..fac069b7eef 100644
--- a/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out
+++ b/iceberg/iceberg-handler/src/test/results/positive/alter_table_to_iceberg.q.out
@@ -484,3 +484,60 @@ POSTHOOK: query: drop table tbl_avro
 POSTHOOK: type: DROPTABLE
 POSTHOOK: Input: default@tbl_avro
 POSTHOOK: Output: default@tbl_avro
+PREHOOK: query: drop table if exists part_tbl_parquet
+PREHOOK: type: DROPTABLE
+POSTHOOK: query: drop table if exists part_tbl_parquet
+POSTHOOK: type: DROPTABLE
+PREHOOK: query: create external table part_tbl_parquet (a int) partitioned by (s string) stored as parquet
+PREHOOK: type: CREATETABLE
+PREHOOK: Output: database:default
+PREHOOK: Output: default@part_tbl_parquet
+POSTHOOK: query: create external table part_tbl_parquet (a int) partitioned by (s string) stored as parquet
+POSTHOOK: type: CREATETABLE
+POSTHOOK: Output: database:default
+POSTHOOK: Output: default@part_tbl_parquet
+PREHOOK: query: insert into part_tbl_parquet partition (s) values (1, '2023/05/18')
+PREHOOK: type: QUERY
+PREHOOK: Input: _dummy_database@_dummy_table
+PREHOOK: Output: default@part_tbl_parquet
+POSTHOOK: query: insert into part_tbl_parquet partition (s) values (1, '2023/05/18')
+POSTHOOK: type: QUERY
+POSTHOOK: Input: _dummy_database@_dummy_table
+POSTHOOK: Output: default@part_tbl_parquet
+POSTHOOK: Output: default@part_tbl_parquet@s=2023%2F05%2F18
+POSTHOOK: Lineage: part_tbl_parquet PARTITION(s=2023/05/18).a SCRIPT []
+PREHOOK: query: select * from part_tbl_parquet
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_tbl_parquet
+PREHOOK: Input: default@part_tbl_parquet@s=2023%2F05%2F18
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from part_tbl_parquet
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_tbl_parquet
+POSTHOOK: Input: default@part_tbl_parquet@s=2023%2F05%2F18
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1	2023/05/18
+PREHOOK: query: alter table part_tbl_parquet convert to iceberg
+PREHOOK: type: ALTERTABLE_CONVERT
+PREHOOK: Input: default@part_tbl_parquet
+POSTHOOK: query: alter table part_tbl_parquet convert to iceberg
+POSTHOOK: type: ALTERTABLE_CONVERT
+POSTHOOK: Input: default@part_tbl_parquet
+POSTHOOK: Output: default@part_tbl_parquet
+PREHOOK: query: select * from part_tbl_parquet
+PREHOOK: type: QUERY
+PREHOOK: Input: default@part_tbl_parquet
+PREHOOK: Output: hdfs://### HDFS PATH ###
+POSTHOOK: query: select * from part_tbl_parquet
+POSTHOOK: type: QUERY
+POSTHOOK: Input: default@part_tbl_parquet
+POSTHOOK: Output: hdfs://### HDFS PATH ###
+1	2023/05/18
+PREHOOK: query: drop table part_tbl_parquet
+PREHOOK: type: DROPTABLE
+PREHOOK: Input: default@part_tbl_parquet
+PREHOOK: Output: default@part_tbl_parquet
+POSTHOOK: query: drop table part_tbl_parquet
+POSTHOOK: type: DROPTABLE
+POSTHOOK: Input: default@part_tbl_parquet
+POSTHOOK: Output: default@part_tbl_parquet