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

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

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

   
   > @parthchandra Would you mind having a look at my I/O performance 
optimization plan for ParquetMR? I think we should coordinate, since we have 
some ideas that might overlap what we touch.
   > 
https://docs.google.com/document/d/1fBGpF_LgtfaeHnPD5CFEIpA2Ga_lTITmFdFIcO9Af-g/edit?usp=sharing
   
   @theosib-amazon I read your document and went thru  #960. It looks like for 
the most part, #960 and this PR and complement each other.  The overlap I see 
is in the changes to `MultiBufferInputStream` where you have added the 
`readFully`, and `skipFully` APIs. The bulk of my changes for async IO are in a 
class derived from `MultiBufferInputStream` and the heart of the changes 
depends on overriding `MultiBufferInputStream.nextBuffer`. In 
`MultiBufferInputStream.nextBuffer` the assumption is that all the buffers have 
been read into. In `AsyncMultiBufferInputStream.nextBuffer` this assumption is 
removed and the call *blocks* only if the next required buffer has not been 
read into.
   Now, `skipFully` and `readFully` are potentially blocking calls because both 
call `nextBuffer` repeatedly if necessary. To gain  maximum pipelining, you 
want to make calls to skipFully and readFully such that you never block for too 
long (or at all) in the call. You will get this if you are skipping or reading 
less than the number of bytes in a single buffer. This is generally the case as 
decompression and decoding is at the page level and that is smaller than the 
size of a single buffer. However, for your optimizations, you should be aware 
of this behaviour.
   From what I see, I don't think there will be a conflict.  I'll pull in your 
PR and give it a deeper look. 
   




> 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