[ https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17552331#comment-17552331 ]
ASF GitHub Bot commented on PARQUET-2149: ----------------------------------------- steveloughran commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r893760667 ########## parquet-common/src/main/java/org/apache/parquet/bytes/SequenceByteBufferInputStream.java: ########## @@ -0,0 +1,269 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.parquet.bytes; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.ArrayList; +import java.util.Collection; +import java.util.Collections; +import java.util.Iterator; +import java.util.List; + +/** + * A bare minimum implementation of a {@link java.io.SequenceInputStream} that wraps an + * <i>ordered</i> collection of ByteBufferInputStreams. + * <p> + * This class, as implemented, is intended only for a specific use in the ParquetFileReader and + * throws {@link UnsupportedOperationException} in unimplemented methods to catch any unintended + * use in other cases. + * <p> + * Even thought this class is derived from ByteBufferInputStream it explicitly does not support any + * byte buffer related methods like slice. It does, however support sliceBuffers which is a + * curious case of reading data from underlying streams + * <p> + * Even though this class changes the state of the underlying streams (by reading from them) + * it does not own them and so the close method does not close the streams. To avoid resource + * leaks the calling code should close the underlying streams + */ +public class SequenceByteBufferInputStream extends ByteBufferInputStream { + + Collection<ByteBufferInputStream> collection; + Iterator<ByteBufferInputStream> iterator; + ByteBufferInputStream current; + long position = 0; + + @Override + public String toString() { + return "SequenceByteBufferInputStream{" + + "collection=" + collection + + ", current=" + current + + ", position=" + position + + '}'; + } + + public SequenceByteBufferInputStream(Collection<ByteBufferInputStream> collection) { + this.collection = collection; + iterator = collection.iterator(); + current = iterator.hasNext() ? iterator.next() : null; + if (current == null) { + throw new UnsupportedOperationException( + "Initializing SequenceByteBufferInputStream with an empty collection is not supported"); + } + } + + @Override + public long position() { + return position; + } + + @Override + public int read(ByteBuffer out) { Review Comment: good to know. > 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)