[ https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17635497#comment-17635497 ]
ASF GitHub Bot commented on PARQUET-2149: ----------------------------------------- parthchandra commented on PR #968: URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1319040697 @wgtmac thank you for looking at this. I don't have any more TODOs on this PR. > * Adopt the incoming Hadoop vectored io api. This should be part of another PR. There is a draft PR (#999 ) open for this. Once that is merged in, I can revisit the async I/O code and incorporate the vectored io api. In other experiments I have seen that async io gives better results over slower networks. With faster network connections, as is the case where we are reading from S3 within an AWS environment, reading in parallel (as the vector io api does), starts to give better results. I believe, that both should be available as options. > * Benchmark against remote object stores from different cloud providers. The numbers I posted earlier were for reading from AWS/S3 over a 1 Gbps line. Reading from within AWS shows lesser improvement. I don't have an account with other cloud providers. Any help here would be appreciated. > IMO, switching `ioThreadPool` and `processThreadPool` the reader instance level will make it more flexible. I've changed the thread pool so that it is not initialized by default but I left them as static members. Ideally, there should be a single IO thread pool that handles all the IO for a process and the size of the pool is determined by the bandwidthof the underlying storage system. Making them per instance is not an issue though. The calling code can decide to set the same thread pool for all instances and achieve the same result. Let me update this. Also, any changes you want to make are fine with me, and the help is certainly appreciated ! > 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)