-
Notifications
You must be signed in to change notification settings - Fork 2.4k
New issue
Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.
By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.
Already on GitHub? Sign in to your account
[SUPPORT] S3 slow file listing causes Hudi read performance. #1829
Comments
@zuyanton this seems like a general issue with
can you try adding cc @n3nash IIRC you mentioned a similar approach done at uber? |
@vinothchandar it didnt have any effect and I believe it shouldn't, since from what it looks like that parameter only gives improvement if you are trying to list statuses of multiple dirs https://github.com/apache/hadoop/blob/trunk/hadoop-mapreduce-project/hadoop-mapreduce-client/hadoop-mapreduce-client-core/src/main/java/org/apache/hadoop/mapred/FileInputFormat.java#L216 where is in our case its always one dir - the root location of single partition. |
I think the finding by @zuyanton seems correct. Increasing the Another thing we can potentially explore is using Spark to perform this listing parallely on the cluster. But this seems like something we should target for |
@zuyanton : This sounds like a general Spark/HMS query integration issue. Are we seeing similar behavior when running the same query over non-hudi table ? |
@bvaradar we dont see similar issue with regular non hudi tables saved to s3 in parquet format. for regular tables "overhead" is the same and under one minute despite the number of partitions. Regular tables with 20k partitions as well as 100 partition take the same time to "load" before spark starts running its jobs where is hudi table on s3 becomes slow with 5k+ partitions. Although we use EMR 5.28 which comes with EMRFS s3 optimized committer enabled in spark by default ,so I assume whatever bottlenecks s3 has, are addressed in the committer. |
Thanks @zuyanton for the updates. IIUC, S3 optimized committer was for optimizing writes reducing the renames done. I might be wrong but I am generally curious on EMR optimizations for Spark. @umehrot2 : We can look at the option you mentioned regarding setting the partition paths and then increasing the num-threads. Is this one of the optimizations done internally within EMR spark ? |
@zuyanton In your test with regular parquet tables you are probably not setting the following property in the spark config However the way Hudi works is it uses But @bvaradar irrespective I think for Hudi we should always compare our performance against standard spark performance (native listing and reading) and not the performance of spark when it is made to go through InputFormat. So we need to get this fixed either ways if we have to be comparable to spark parquet performance which uses parallelized listing over the cluster. |
@bvaradar @zuyanton EMR S3 optimized committer only helps avoid renames. Again that does not come into effect for Hudi because of the way Hudi datasource is implemented. Hudi datasource is not an extension of Irrespective the committer would not have any effect on this listing performance. |
@umehrot2 you are right , with |
@rubenssoto for some code paths, it will be. if you turn on |
Thank you so much for your answer. Is this configuration right?
I made these 2 queries: spark.read.format('hudi').load('s3://ze-data-lake/temp/order_test').count() %%sql On the pyspark query spark creates a job with 143 tasks, after 10 seconds of listing the count was fast, but in the spark sql query spark creates a job with 2000 tasks and was very slow, is it a Hudi or spark issue? Another problem that I got it, my table has 36 million rows, with that config shows only 4 million. |
0.7.0 is being voted on right now. Hopefully today. So the |
0.7.0 is out! |
With 0.7.0, one can set |
Hudi MoR reading performance gets slower on tables with many (1000+) partitions stored in S3. When running simple
spark.sql("select * from table_ro).count
command, we observe in spark UI that first 2.5 minutes no spark jobs gets scheduled and the load on cluster during that period is almost non existing.When looking into logs to figure out what is going on during that period we observe that first two and a half minutes Hudi is busy running
HoodieParquetInputFormat.listStatus
code link. I placed timer logs lines around various parts of that function and was able to narrow down to this linehudi/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieParquetInputFormat.java
Line 103 in f5dc8ca
If I understand correctly what this line does it lists all files in a single partition.
Looks like this "overhead" is linearly depends on number of partitions as increasing number of partitions to 2000 almost doubles the overhead and cluster just runs
HoodieParquetInputFormat.listStatus
before starting executing any spark jobs.To Reproduce
see code snippet bellow
Hudi version : master branch
Spark version : 2.4.4
Hive version : 2.3.6
Hadoop version : 2.8.5
Storage (HDFS/S3/GCS..) : S3
Running on Docker? (yes/no) : no
Additional context
The text was updated successfully, but these errors were encountered: