You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues-all@impala.apache.org by "Quanlong Huang (Jira)" <ji...@apache.org> on 2020/06/01 02:50:00 UTC

[jira] [Commented] (IMPALA-9805) Unnecessary reloading partitions with inconsistent name strings between Impala and Hive

    [ https://issues.apache.org/jira/browse/IMPALA-9805?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17120727#comment-17120727 ] 

Quanlong Huang commented on IMPALA-9805:
----------------------------------------

I looked into HMS's db. HMS stores the partition names and respects the location.
{code}
$ psql -q -U hiveuser -d HMS_home_quanlong_workspace_Impala_cdp
HMS_home_quanlong_workspace_Impala_cdp=> select p."PART_NAME" from "TBLS" t join "PARTITIONS" p using ("TBL_ID") where "TBL_NAME" = 'hudi_partitioned';
         PART_NAME         
---------------------------
 year=2015/month=03/day=16
 year=2015/month=03/day=17
 year=2016/month=03/day=15
(3 rows)
{code}

> Unnecessary reloading partitions with inconsistent name strings between Impala and Hive
> ---------------------------------------------------------------------------------------
>
>                 Key: IMPALA-9805
>                 URL: https://issues.apache.org/jira/browse/IMPALA-9805
>             Project: IMPALA
>          Issue Type: Bug
>          Components: Catalog
>            Reporter: Quanlong Huang
>            Assignee: Quanlong Huang
>            Priority: Critical
>
> For AlterTable Recover Partitions, Impala compares the Hive partition names with the names of cached partitions to drop non-existing partitions and add new partitions. However, comparing by name strings is not enough since some partitions will have inconsistent names between Hive and Impala. This usually happens when the partition directory is created by non-Hive apps.
> In the same scenario, REFRESH will reload these partitions twice, because they are added into both "removedPartitions" and "partitionsToLoadFiles". Code snipper:
> {code:java}
>     for (HdfsPartition partition: partitionMap_.values()) {
>       // Remove partitions that don't exist in the Hive Metastore. These are partitions
>       // that were removed from HMS using some external process, e.g. Hive.
>       if (!msPartitionNames.contains(partition.getPartitionName())) {
>         removedPartitions.add(partition);       <------------- Add it here
>       }
>       if (partition.isDirty()) {
>         // Dirty partitions are updated by removing them from table's partition
>         // list and loading them from the Hive Metastore.
>         dirtyPartitions.add(partition);
>       } else {
>         if (partitionsToUpdate == null && loadPartitionFileMetadata) {
>           partitionsToLoadFiles.add(partition);          <-------- Also add it here
>         }
>       }
>       Preconditions.checkNotNull(partition.getCachedMsPartitionDescriptor());
>       partitionNames.add(partition.getPartitionName());
>     }
> {code}
> [https://github.com/apache/impala/blob/56ee90c598dcc637f10647ffc3e03cc0a70b92ce/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java#L1176]
> So they are loaded as new partitions and loaded again as "partitionsToLoadFiles" in these two calls:
>  * [https://github.com/apache/impala/blob/56ee90c598dcc637f10647ffc3e03cc0a70b92ce/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java#L1204]
>  * [https://github.com/apache/impala/blob/56ee90c598dcc637f10647ffc3e03cc0a70b92ce/fe/src/main/java/org/apache/impala/catalog/HdfsTable.java#L1224]
>  
> Let's say external table my_part_tbl is partitioned by (year int, month {color:#de350b}*int*{color}, day {color:#de350b}*int*{color}). User creates and uploads data to HDFS dir year=2020/month={color:#ff0000}*01*{color}/day={color:#ff0000}*01*{color}{color:#172b4d}, and then triggers an AlterTable RecoverPartitions command in Impala. Impala will create partition (year=2020/month=*1*{color}/day=*1*) in Hive using this location ".../year=2020/month=*{color:#de350b}01{color}*/day=*{color:#de350b}01{color}*".
> Next time when running AlterTable RecoverPartition again (e.g. when new partition dirs are created again), the partition name list got from Hive is [year=2020/month=01/day=01]. However, the name list of cached partitions is [year=2020/month=1/day=1]. Impala will drop this partition and load it as a new partition.
> {color:#172b4d}This impacts the performance of AlterTable RecoverPartition on partitioned tables if the partition directories are all in such case. Many partitions will be reload and reload.{color}
> *{color:#172b4d}Reproduction{color}*
> {code:java}
> impala> create external table my_part_tbl (id int) partitioned by (year int, month int, day int);
> impala> describe formatted my_part_tbl;{code}
> Found the table location is hdfs://localhost:20500/test-warehouse/my_part_tbl
> Create and upload data to a partition dir using HDFS CLI:
> {code:java}
> $ cat data.txt
> 1
> 2
> 3
> $ hdfs dfs -mkdir -p hdfs://localhost:20500/test-warehouse/my_part_tbl/year=2020/month=01/day=01
> $ hdfs dfs -mkdir -p hdfs://localhost:20500/test-warehouse/my_part_tbl/year=2020/month=01/day=02
> $ hdfs dfs -mkdir -p hdfs://localhost:20500/test-warehouse/my_part_tbl/year=2020/month=09/day=01
> $ hdfs dfs -mkdir -p hdfs://localhost:20500/test-warehouse/my_part_tbl/year=2020/month=09/day=02
> $ hdfs dfs -put data.txt hdfs://localhost:20500/test-warehouse/my_part_tbl/year=2020/month=01/day=01
> $ hdfs dfs -put data.txt hdfs://localhost:20500/test-warehouse/my_part_tbl/year=2020/month=01/day=02
> $ hdfs dfs -put data.txt hdfs://localhost:20500/test-warehouse/my_part_tbl/year=2020/month=09/day=01
> $ hdfs dfs -put data.txt hdfs://localhost:20500/test-warehouse/my_part_tbl/year=2020/month=09/day=02{code}
> Let Impala detect these partitions.
> {code:java}
> impala> alter table my_part_tbl recover partitions;
> {code}
> Then everytime when running AlterTable RecoverPartitions, these 4 partitions will be reloaded again. The logs of catalogd reflects this:
> {code:java}
> I0531 11:51:45.037181 27878 HdfsTable.java:1001] Reloading metadata for all partition(s) of default.my_part_tbl (ALTER TABLE RECOVER_PARTITIONS)
> I0531 11:51:45.037286 27878 HdfsTable.java:2095] Load Valid Write Id List Done. Time taken: 4.456us
> I0531 11:51:45.049304 27878 ParallelFileMetadataLoader.java:144] Loading file and block metadata for 4 paths for table default.my_part_tbl using a thread pool of size 4
> I0531 11:51:45.053562 27878 HdfsTable.java:697] Loaded file and block metadata for default.my_part_tbl partitions: year=2020/month=1/day=1, year=2020/month=1/day=2, year=2020/month=9/day=1, and 1 others. Time taken: 4.537ms
> I0531 11:51:45.053689 27878 HdfsTable.java:1032] Incrementally loaded table metadata for: default.my_part_tbl
> I0531 11:51:45.279531 26899 catalog-server.cc:735] Collected update: 1:TABLE:default.my_part_tbl, version=1512, original size=3989, compressed size=1294
> I0531 11:51:45.281633 26899 catalog-server.cc:735] Collected update: 1:CATALOG_SERVICE_ID, version=1512, original size=60, compressed size=58
> I0531 11:51:47.277819 26906 catalog-server.cc:340] A catalog update with 2 entries is assembled. Catalog version: 1512 Last sent catalog version: 1511
> {code}
> Running REFRESH on this table will see the partitions being reloaded twice.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)

---------------------------------------------------------------------
To unsubscribe, e-mail: issues-all-unsubscribe@impala.apache.org
For additional commands, e-mail: issues-all-help@impala.apache.org