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

ASF GitHub Bot commented on PARQUET-2149:
-----------------------------------------

steveloughran commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1134843705

   1. whose s3 client was used for testing here -if the s3a one, which hadoop 
release?
   2. the azure abfs and gcs connectors do async prefetching of the next block, 
but are simply assuming that code will read sequentially; if there is another 
seek/readFully to a new location, those prefetches will be abandoned. there is 
work in s3a to do prefetching here with caching, so as to reduce the penalty of 
backwards seeks. https://issues.apache.org/jira/browse/HADOOP-18028
   
   hadoop is adding a vectored IO api intended for libraries like orc and 
parquet to be able to use, where the application provides an unordered list of 
ranges, a bytebuffer supplier and gets back a list of futures to wait for. the 
base implementation simply reads using readFully APi. s3a (and later abfs) will 
do full async retrieval itself, using the http connection pool.
    https://issues.apache.org/jira/browse/HADOOP-18103
   
   both vectored io and s3a prefetching will ship this summer in hadoop 3.4.0. 
i don't see this change conflicting with this, though they may obsolete a lot 
of it.
   
   have you benchmarked this change with abfs or google gcs connectors to see 
what difference it makes there?




> Implement async IO for Parquet file reader
> ------------------------------------------
>
>                 Key: PARQUET-2149
>                 URL: https://issues.apache.org/jira/browse/PARQUET-2149
>             Project: Parquet
>          Issue Type: Improvement
>          Components: parquet-mr
>            Reporter: Parth Chandra
>            Priority: Major
>
> ParquetFileReader's implementation has the following flow (simplified) - 
>       - For every column -> Read from storage in 8MB blocks -> Read all 
> uncompressed pages into output queue 
>       - From output queues -> (downstream ) decompression + decoding
> This flow is serialized, which means that downstream threads are blocked 
> until the data has been read. Because a large part of the time spent is 
> waiting for data from storage, threads are idle and CPU utilization is really 
> low.
> There is no reason why this cannot be made asynchronous _and_ parallel. So 
> For Column _i_ -> reading one chunk until end, from storage -> intermediate 
> output queue -> read one uncompressed page until end -> output queue -> 
> (downstream ) decompression + decoding
> Note that this can be made completely self contained in ParquetFileReader and 
> downstream implementations like Iceberg and Spark will automatically be able 
> to take advantage without code change as long as the ParquetFileReader apis 
> are not changed. 
> In past work with async io  [Drill - async page reader 
> |https://github.com/apache/drill/blob/master/exec/java-exec/src/main/java/org/apache/drill/exec/store/parquet/columnreaders/AsyncPageReader.java]
>  , I have seen 2x-3x improvement in reading speed for Parquet files.



--
This message was sent by Atlassian Jira
(v8.20.7#820007)

Reply via email to