[
https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17635582#comment-17635582
]
ASF GitHub Bot commented on PARQUET-2149:
-----------------------------------------
parthchandra commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1319411064
> > 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 !
I'm thinking of merging the thread pools into a single ioThreadPool and
making the ioThreadPool settable thru `ParquetReadOptions` (like the allocator
is). The work being done by the processThreadPool is rather small and maybe we
can do away with it.
Adding the pool via `ParquetReadOptions` makes it easier to use with
`ParquetReader` (used a lot in unit tests).
WDYT?
> 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)