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

Udit Mehrotra commented on HUDI-829:
------------------------------------

[~nishith29] Thanks for creating the ticket. So the issue I was talking about 
happens becuase of 
[HoodieROTablePathFilter|https://github.com/apache/incubator-hudi/blob/master/hudi-hadoop-mr/src/main/java/org/apache/hudi/hadoop/HoodieROTablePathFilter.java]
 which we pass to spark to filter out the latest commit files for copy on write 
use-case. When we create the *parquet* datasource underneath for reading, Spark 
uses 
[InMemoryFileIndex|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala]
 to maintain an index with list of files to read from the path passed to the 
data source. Here while the listing of files is parallelized by Spark, the 
filter however is applied sequentially on the all the file one by one 
[here|https://github.com/apache/spark/blob/master/sql/core/src/main/scala/org/apache/spark/sql/execution/datasources/InMemoryFileIndex.scala#L376].
 Here also there are some caveats, if the number of paths to be listed < 
*sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold* Spark 
optimizes to do serial listing and the filters are all evaluated sequentially 
across all partitions at the driver. If however its > 
*sparkSession.sessionState.conf.parallelPartitionDiscoveryThreshold* then 
listing across paths is parallelized and filter is applied sequentially at the 
executors but still sequentially for each file in the path. We have seen this 
as being a bottleneck with S3 atleast. While ideally this is something that can 
improved inside of spark by parallelizing the filtering further, however at 
this point I need to dive further into filter code to understand if there is 
something that can be done to make it faster from Hudi side.

> Efficiently reading hudi tables through spark-shell
> ---------------------------------------------------
>
>                 Key: HUDI-829
>                 URL: https://issues.apache.org/jira/browse/HUDI-829
>             Project: Apache Hudi (incubating)
>          Issue Type: Task
>          Components: Spark Integration
>            Reporter: Nishith Agarwal
>            Assignee: Nishith Agarwal
>            Priority: Major
>
> [~uditme] Created this ticket to track some discussion on read/query path of 
> spark with Hudi tables. 
> My understanding is that when you read Hudi tables through spark-shell, some 
> of your queries are slower due to some sequential activity performed by spark 
> when interacting with Hudi tables (even with 
> spark.sql.hive.convertMetastoreParquet which can give you the same data 
> reading speed and all the vectorization benefits). Is this slowness observed 
> during spark query planning ? Can you please elaborate on this ? 



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

Reply via email to