You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@parquet.apache.org by GitBox <gi...@apache.org> on 2022/05/18 23:03:31 UTC

[GitHub] [parquet-mr] parthchandra commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader

parthchandra commented on code in PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#discussion_r876197602


##########
parquet-column/src/main/java/org/apache/parquet/column/page/PageReader.java:
##########
@@ -37,4 +39,9 @@ public interface PageReader {
    * @return the next page in that chunk or null if after the last page
    */
   DataPage readPage();
+
+  /**
+   * Close the page reader. By default it is no-op.
+   */
+  default void close() throws IOException {}

Review Comment:
   Sure.



##########
parquet-common/src/main/java/org/apache/parquet/bytes/AsyncMultiBufferInputStream.java:
##########
@@ -0,0 +1,173 @@
+/*
+ *  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);
+
+  final SeekableInputStream fileInputStream;
+  int fetchIndex = 0;
+  int readIndex = 0;
+  ExecutorService threadPool;
+  LinkedBlockingQueue<Future<Void>> readFutures;
+  boolean closed = false;
+  Exception ioException;
+
+  LongAdder totalTimeBlocked = new LongAdder();
+  LongAdder totalCountBlocked = new LongAdder();
+  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");
+    }
+    synchronized (this) {
+      if (ioException != null) {
+        throw new RuntimeException(ioException);
+      }
+    }
+  }
+
+  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);
+      long putStart = System.nanoTime();
+      long putCompleted = System.nanoTime();
+      LOG.debug("ASYNC Stream: FS READ (output) BLOCKED - {}",
+        (putCompleted - putStart) / 1000.0);
+      fetchIndex++;
+    } catch (IOException e) {
+      // Save the exception so that the calling thread can check if something went wrong.
+      // checkState will throw an exception if the read task has failed.
+      synchronized(this) {
+        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;
+    }

Review Comment:
   Glad you made me look. The synchronization was needed because I was saving any exception that might have occurred in the reader thread. However, that exception will be captured by the Future object and thrown as an ExecutionException when `Future,get` is called in line 135. 
   So we don't really need to save the exception and the synchronization around it can also be removed.



-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: dev-unsubscribe@parquet.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org