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

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

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

   > Hi, I am very interested in this optimization and just have some questions 
when testing in a cluster with 4nodes/96 cores using spark3.1. Unfortunately, I 
see little improvement.
   
   You're likely to see improvement in cases where file i/o is the bottleneck. 
Most TPC-DS queries are join heavy and you will see little improvement there. 
You might do better with TPC-H. 
   
   > I am confused than whether it is necessary to keep 
spark.sql.parquet.enableVectorizedReader = false in spark when testing with 
spark 3.1 and how can I set the parquet buffer size. 
   
   It's probably best to keep the parquet (read) buffer size untouched.
   
   You should keep `spark.sql.parquet.enableVectorizedReader = true` 
irrespective of this. This feature improves I/O speed of reading raw data. The 
Spark vectorized reader kicks in after data is read from storage and converts 
the raw data into Spark's internal columnar representation and is faster than 
the row based version.  
   




> 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.10#820010)

Reply via email to