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

2022-12-03 Thread GitBox


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


##
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> 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 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:
   I see the comment with 'hack'. What is the proper implementation? 



-- 
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-unsubscr...@parquet.apache.org

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



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

2022-12-03 Thread GitBox


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


##
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> 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 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 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() {
+   

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

2022-12-03 Thread GitBox


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


##
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> 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 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 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() {
+   

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

2022-06-19 Thread GitBox


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


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -46,12 +46,11 @@
 import java.util.List;
 import java.util.Map;
 import java.util.Map.Entry;
+import java.util.Optional;
 import java.util.Set;
-import java.util.concurrent.Callable;
-import java.util.concurrent.ExecutionException;
-import java.util.concurrent.ExecutorService;
-import java.util.concurrent.Executors;
-import java.util.concurrent.Future;
+import java.util.concurrent.*;

Review Comment:
   I guess it is IDE does that but let's not use wildcard here 



##
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

Review Comment:
   I understand we want applications to provide their own implementations, but 
can you share why we choose the cached thread pool instead of fixed in default? 
I kind of feel a lot of user scenarios of Parquet is with unpredictable 
execution times and we need better control over our program's resource 
consumption. 



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -1387,8 +1489,13 @@ public void close() throws IOException {
* result of the column-index based filtering when some pages might be 
skipped at reading.
*/
   private class ChunkListBuilder {
+// ChunkData is backed by either a list of buffers or a list of strams

Review Comment:
   typo? streams? 



##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ParquetFileReader.java:
##
@@ -1796,5 +1882,314 @@ public void readAll(SeekableInputStream f, 
ChunkListBuilder builder) throws IOEx
 public long endPos() {
   return offset + length;
 }
+
+@Override
+public String toString() {
+  return "ConsecutivePartList{" +
+"offset=" + offset +
+", length=" + length +
+", chunks=" + chunks +
+'}';
+}
   }
+
+  /**
+   * Encapsulates the reading of a single page.
+   */
+  public class PageReader implements Closeable {
+private final Chunk chunk;
+private final int currentBlock;
+private final BlockCipher.Decryptor headerBlockDecryptor;
+private final BlockCipher.Decryptor pageBlockDecryptor;
+private final byte[] aadPrefix;
+private final int rowGroupOrdinal;
+private final int columnOrdinal;
+
+//state
+private final LinkedBlockingDeque> pagesInChunk = new 
LinkedBlockingDeque<>();
+private DictionaryPage dictionaryPage = null;
+private int pageIndex = 0;
+private long valuesCountReadSoFar = 0;
+private int dataPageCountReadSoFar = 0;
+
+// derived
+private final PrimitiveType type;
+private final byte[] dataPageAAD;
+private final byte[] dictionaryPageAAD;
+private byte[] dataPageHeaderAAD = null;
+
+private final BytesInputDecompressor decompressor;
+
+private final ConcurrentLinkedQueue> readFutures = new 
ConcurrentLinkedQueue<>();
+
+private final LongAdder totalTimeReadOnePage = new LongAdder();
+private final LongAdder totalCountReadOnePage = new LongAdder();
+private final LongAccumulator maxTimeReadOnePage = new 
LongAccumulator(Long::max, 0L);
+private final LongAdder totalTimeBlockedPagesInChunk = new LongAdder();
+private final LongAdder totalCountBlockedPagesInChunk = new LongAdder();
+private final LongAccumulator maxTimeBlockedPagesInChunk = new 
LongAccumulator(Long::max, 0L);
+
+public PageReader(Chunk chunk, int currentBlock, Decryptor 
headerBlockDecryptor,
+  Decryptor pageBlockDecryptor, byte[] aadPrefix, int rowGroupOrdinal, int 
columnOrdinal,
+  BytesInputDecompressor decompressor
+  ) {
+  this.chunk = chunk;
+  this.currentBlock = currentBlock;
+  this.headerBlockDecryptor = headerBlockDecryptor;
+  this.pageBlockDecryptor = pageBlockDecryptor;
+  this.aadPrefix = aadPrefix;
+  this.rowGroupOrdinal = rowGroupOrdinal;
+  this.columnOrdinal = columnOrdinal;
+  this.decompressor = decompressor;
+
+  this.type = getFileMetaData().getSchema()
+.getType(chunk.descriptor.col.getPath()).asPrimitiveType();
+
+  if (null != headerBlockDecryptor) {
+dataPageHeaderAAD = AesCipher.createModuleAAD(aadPrefix, 
ModuleType.DataPageHeader,
+  rowGroupOrdinal,
+  columnOrdinal, chunk.getPageOrdinal(dataPageCountReadSoFar));
+  }
+  if (null != 

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

2022-06-18 Thread GitBox


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


##
parquet-hadoop/src/main/java/org/apache/parquet/HadoopReadOptions.java:
##
@@ -61,9 +65,10 @@ private HadoopReadOptions(boolean useSignedStringMinMax,
 Configuration conf,
 FileDecryptionProperties fileDecryptionProperties) 
{
 super(
-useSignedStringMinMax, useStatsFilter, useDictionaryFilter, 
useRecordFilter, useColumnIndexFilter,
-usePageChecksumVerification, useBloomFilter, recordFilter, 
metadataFilter, codecFactory, allocator,
-maxAllocationSize, properties, fileDecryptionProperties
+  useSignedStringMinMax, useStatsFilter, useDictionaryFilter, 
useRecordFilter,

Review Comment:
   it seems two spaces were 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-unsubscr...@parquet.apache.org

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