[ https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17557373#comment-17557373 ]
ASF GitHub Bot commented on PARQUET-2149: ----------------------------------------- ggershinsky commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r903592957 ########## parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java: ########## @@ -126,6 +127,42 @@ public class ParquetFileReader implements Closeable { public static String PARQUET_READ_PARALLELISM = "parquet.metadata.read.parallelism"; + public static int numProcessors = Runtime.getRuntime().availableProcessors(); + + // Thread pool to read column chunk data from disk. Applications should call setAsyncIOThreadPool + // to initialize this with their own implementations. + // Default initialization is useful only for testing + public static ExecutorService ioThreadPool = Executors.newCachedThreadPool( + r -> new Thread(r, "parquet-io")); + + // Thread pool to process pages for multiple columns in parallel. Applications should call + // setAsyncProcessThreadPool to initialize this with their own implementations. + // Default initialization is useful only for testing + public static ExecutorService processThreadPool = Executors.newCachedThreadPool( Review Comment: given the comment "Default initialization is useful only for testing", maybe this can be moved to the tests? > 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)