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

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

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

   I have some numbers from an internal benchmark using Spark. I didn't see any 
benchmarks in the Parquet codebase that I could reuse.
   
   Here are the numbers from my own benchmark - 
   
    - 10 runs, each run reads all columns from store_sales (the largest table) 
in the TPC-DS (100G) dataset
     `spark.sql("select * from store_sales")`
     - Sync reader with default 8MB buffer size, Async reader with 1MB buffer 
size (achieves better pipelining)
     - Run on Macbook Pro, reading from S3. Spark has 6 cores.
     - All times in seconds
      
     | Run | Async | Sync | Async (w/o outliers)| Sync (w/o outliers) |
     | ---:| ---:| ---:| ---:| ---:| 
   |1| 84| 102| - | - |         
   |2| 90| 366| 90| 366|
   |3| 78| 156| - | 156|
   |4| 84| 128| 84| - |
   |5| 108|402| - | - |         
   |6| 90| 432| 90| - |
   |7| 84| 378| 84| 378|
   |8| 108|324| - | 324|
   |9| 90| 318| 90| 318|
   |10|90| 282| 90| 282|
   |Average| 90.6|      288.8| 88| 304|
   |Median| 90| 321| **90**| **321**|    
   |StdDev| 9.98| 119.
   
   
   After removing the two highest and two lowest runs for each case, and taking 
the median value:
   
   Async: 90 sec
   Sync: 321 sec
   
   




> 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