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

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

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

   > My test is spark.sql("select * from store_sales order by ss_customer_sk 
limit 10"), store_sales is table of 1TB TP-CDS.
   
   Parquet-io  and parquet-process  threads is hardcoded given like this in 
ParquetReadOptions.java
   public static class Builder {
       protected ExecutorService ioThreadPool = Executors.newFixedThreadPool(4);
       protected ExecutorService processThreadPool = 
Executors.newFixedThreadPool(4);
   }
   
   I also take the follow test with local filesystem using 100GB TP-CDS 
store_sales table,and I see there is a degradation with async io feature. 
   test("parquet reader select") {
       val sc = SparkSession.builder().master("local[4]").getOrCreate()
       val df = sc.read.parquet("file:///D:\\work\\test\\tpcds\\store_sales")
       df.createOrReplaceTempView("table")
       val start = System.currentTimeMillis()
       sc.sql("select * from table order by ss_customer_sk limit 10").show()
       val end = System.currentTimeMillis()
       System.out.println("time: " + (end - start))
     }
   without this feature -> time: 7240
   with this feature -> time: 19923
   
   Threads are as expected 
   
![image](https://user-images.githubusercontent.com/87682445/221344885-a49cc5f2-9eba-4f0d-bc06-7615416d5b02.png)
   
   What process did I go wrong and can you show me the correct way to use this 
feature?
   
      
   
   
   
   
    
   




> 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