[ https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17643502#comment-17643502 ]
ASF GitHub Bot commented on PARQUET-2149: ----------------------------------------- parthchandra commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r1039934636 ########## parquet-common/src/main/java/org/apache/parquet/bytes/AsyncMultiBufferInputStream.java: ########## @@ -0,0 +1,162 @@ +/* + * 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.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.LongAccumulator; +import java.util.concurrent.atomic.LongAdder; +import org.apache.parquet.io.SeekableInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class AsyncMultiBufferInputStream extends MultiBufferInputStream { + + private static final Logger LOG = LoggerFactory.getLogger(AsyncMultiBufferInputStream.class); + + private int fetchIndex = 0; + private final SeekableInputStream fileInputStream; + private int readIndex = 0; + private ExecutorService threadPool; + private LinkedBlockingQueue<Future<Void>> readFutures; + private boolean closed = false; + + private LongAdder totalTimeBlocked = new LongAdder(); + private LongAdder totalCountBlocked = new LongAdder(); + private LongAccumulator maxTimeBlocked = new LongAccumulator(Long::max, 0L); + + AsyncMultiBufferInputStream(ExecutorService threadPool, SeekableInputStream fileInputStream, + List<ByteBuffer> buffers) { + super(buffers); + this.fileInputStream = fileInputStream; + this.threadPool = threadPool; + readFutures = new LinkedBlockingQueue<>(buffers.size()); + if (LOG.isDebugEnabled()) { + LOG.debug("ASYNC: Begin read into buffers "); + for (ByteBuffer buf : buffers) { + LOG.debug("ASYNC: buffer {} ", buf); + } + } + fetchAll(); + } + + private void checkState() { + if (closed) { + throw new RuntimeException("Stream is closed"); + } + } + + private void fetchAll() { + checkState(); + submitReadTask(0); + } + + private void submitReadTask(int bufferNo) { + ByteBuffer buffer = buffers.get(bufferNo); + try { + readFutures.put(threadPool.submit(() -> { + readOneBuffer(buffer); + if (bufferNo < buffers.size() - 1) { + submitReadTask(bufferNo + 1); + } + return null; + }) + ); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + private void readOneBuffer(ByteBuffer buffer) { + long startTime = System.nanoTime(); + try { + fileInputStream.readFully(buffer); + buffer.flip(); + long readCompleted = System.nanoTime(); + long timeSpent = readCompleted - startTime; + LOG.debug("ASYNC Stream: READ - {}", timeSpent / 1000.0); + fetchIndex++; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean nextBuffer() { + checkState(); + // hack: parent constructor can call this method before this class is fully initialized. + // Just return without doing anything. + if (readFutures == null) { + return false; + } + if (readIndex < buffers.size()) { + long start = System.nanoTime(); + try { + if (LOG.isDebugEnabled()) { + LOG.debug("ASYNC (next): Getting next buffer"); + } + Future<Void> future = readFutures.take(); + future.get(); + long timeSpent = System.nanoTime() - start; + totalCountBlocked.add(1); + totalTimeBlocked.add(timeSpent); + maxTimeBlocked.accumulate(timeSpent); + if (LOG.isDebugEnabled()) { + LOG.debug("ASYNC (next): {}: Time blocked for read {} ns", this, timeSpent); + } + } catch (Exception e) { + if (e instanceof InterruptedException) { + Thread.currentThread().interrupt(); + } + LOG.error("Async (next): exception while getting next buffer: ", e); + throw new RuntimeException(e); + } + readIndex++; + } + return super.nextBuffer(); + } + + public void close() { + LOG.debug("ASYNC Stream: Blocked: {} {} {}", totalTimeBlocked.longValue() / 1000.0, + totalCountBlocked.longValue(), maxTimeBlocked.longValue() / 1000.0); + Future<Void> readResult; + while (!readFutures.isEmpty()) { + try { + readResult = readFutures.poll(); + readResult.get(); + if (!readResult.isDone() && !readResult.isCancelled()) { + readResult.cancel(true); + } else { + readResult.get(1, TimeUnit.MILLISECONDS); + } + } catch (Exception e) { + // Do nothing Review Comment: It's not really useful to log anything here because there is nothing to be done. Some of the calls in the try block throw exceptions and we catch them to prevent them from propagating. We are shutting down and cleaning up the futures so it doesn't matter if there was an exception or not. ########## parquet-common/src/main/java/org/apache/parquet/bytes/AsyncMultiBufferInputStream.java: ########## @@ -0,0 +1,162 @@ +/* + * 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.IOException; +import java.nio.ByteBuffer; +import java.util.List; +import java.util.concurrent.ExecutorService; +import java.util.concurrent.Future; +import java.util.concurrent.LinkedBlockingQueue; +import java.util.concurrent.TimeUnit; +import java.util.concurrent.atomic.LongAccumulator; +import java.util.concurrent.atomic.LongAdder; +import org.apache.parquet.io.SeekableInputStream; +import org.slf4j.Logger; +import org.slf4j.LoggerFactory; + +class AsyncMultiBufferInputStream extends MultiBufferInputStream { + + private static final Logger LOG = LoggerFactory.getLogger(AsyncMultiBufferInputStream.class); + + private int fetchIndex = 0; + private final SeekableInputStream fileInputStream; + private int readIndex = 0; + private ExecutorService threadPool; + private LinkedBlockingQueue<Future<Void>> readFutures; + private boolean closed = false; + + private LongAdder totalTimeBlocked = new LongAdder(); + private LongAdder totalCountBlocked = new LongAdder(); + private LongAccumulator maxTimeBlocked = new LongAccumulator(Long::max, 0L); + + AsyncMultiBufferInputStream(ExecutorService threadPool, SeekableInputStream fileInputStream, + List<ByteBuffer> buffers) { + super(buffers); + this.fileInputStream = fileInputStream; + this.threadPool = threadPool; + readFutures = new LinkedBlockingQueue<>(buffers.size()); + if (LOG.isDebugEnabled()) { + LOG.debug("ASYNC: Begin read into buffers "); + for (ByteBuffer buf : buffers) { + LOG.debug("ASYNC: buffer {} ", buf); + } + } + fetchAll(); + } + + private void checkState() { + if (closed) { + throw new RuntimeException("Stream is closed"); + } + } + + private void fetchAll() { + checkState(); + submitReadTask(0); + } + + private void submitReadTask(int bufferNo) { + ByteBuffer buffer = buffers.get(bufferNo); + try { + readFutures.put(threadPool.submit(() -> { + readOneBuffer(buffer); + if (bufferNo < buffers.size() - 1) { + submitReadTask(bufferNo + 1); + } + return null; + }) + ); + } catch (InterruptedException e) { + Thread.currentThread().interrupt(); + throw new RuntimeException(e); + } + } + + private void readOneBuffer(ByteBuffer buffer) { + long startTime = System.nanoTime(); + try { + fileInputStream.readFully(buffer); + buffer.flip(); + long readCompleted = System.nanoTime(); + long timeSpent = readCompleted - startTime; + LOG.debug("ASYNC Stream: READ - {}", timeSpent / 1000.0); + fetchIndex++; + } catch (IOException e) { + throw new RuntimeException(e); + } + } + + @Override + public boolean nextBuffer() { + checkState(); + // hack: parent constructor can call this method before this class is fully initialized. Review Comment: The fix would be to change behavior of the parent class. Basically the parent class implicitly assumes that there will be no derived classes and called a private method `nextBuffer` in the constructor. I extended the class and made next buffer protected so it is no longer correct to call it in the constructor. Changing the parent class behavior broke too many unit tests. I could have fixed the unit tests as well, but wasn't sure that downstream projects might be depending on the behavior. In the end, it seemed that handling this in the derived class was a safer approach. It's really a workaround to make things easier to implement. Perhaps I should not have called it a hack. > 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)