[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
ggershinsky commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r946662874 ## 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: not sure; looks like many tests use copy/paste, rather than extension.. -- 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] ggershinsky commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
ggershinsky commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r903697361 ## 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; Review Comment: probably not needed -- 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] ggershinsky commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
ggershinsky commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r903693351 ## 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 != pageBlockDecryptor) { +dataPageAAD = AesCipher.createModuleAAD(aadPrefix, ModuleType.DataPage, rowGroupOrdinal, + columnOrdinal, 0); +dictionaryPageAAD = AesCipher.createModuleAAD(aadPrefix, ModuleType.DictionaryPage, Review Comment: Yep, the `dictionaryPageAAD` is not necessary here. This is a significant code change, more than just moving the current logic of ```java public ColumnChunkPageReader readAllPages(BlockCipher.Decryptor headerBlockDecryptor, BlockCipher.Decryptor pageBlockDecryptor, byte[] aadPrefix, int rowGroupOrdinal, int columnOrdinal) ``` I'll have a closer look at the details, but we need a unitest (proposed in my other comment) to make sure decryption works ok with async io and parallel column reading. -- 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] ggershinsky commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
ggershinsky commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r903595526 ## 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 { Review Comment: maybe can also be separated from the ParquetFileReader, this is a chance to reduce the size of the latter :) -- 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] ggershinsky commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
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? -- 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] ggershinsky commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
ggershinsky commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r903469988 ## parquet-hadoop/src/main/java/org/apache/parquet/crypto/InternalFileDecryptor.java: ## @@ -61,10 +61,7 @@ public InternalFileDecryptor(FileDecryptionProperties fileDecryptionProperties) private BlockCipher.Decryptor getThriftModuleDecryptor(byte[] columnKey) { if (null == columnKey) { // Decryptor with footer key - if (null == aesGcmDecryptorWithFooterKey) { -aesGcmDecryptorWithFooterKey = ModuleCipherFactory.getDecryptor(AesMode.GCM, footerKey); - } - return aesGcmDecryptorWithFooterKey; + return ModuleCipherFactory.getDecryptor(AesMode.GCM, footerKey); Review Comment: could you add a unitest of decryption with async io and parallel column reader, eg to the https://github.com/apache/parquet-mr/blob/master/parquet-hadoop/src/test/java/org/apache/parquet/crypto/TestPropertiesDrivenEncryption.java#L507 ## 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: should we be creating thread pools if the Async IO and parallel column reading are not activated? (here and in the line 135) ## 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 { Review Comment: we already have a PageReader (interface). Could you rename this class. ## parquet-common/src/main/java/org/apache/parquet/bytes/AsyncMultiBufferInputStream.java: ## @@ -0,0 +1,158 @@ +/* + * 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, SeekableInputSt
[GitHub] [parquet-mr] ggershinsky commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
ggershinsky commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r903403821 ## 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 != pageBlockDecryptor) { +dataPageAAD = AesCipher.createModuleAAD(aadPrefix, ModuleType.DataPage, rowGroupOrdinal, + columnOrdinal, 0); +dictionaryPageAAD = AesCipher.createModuleAAD(aadPrefix, ModuleType.DictionaryPage, Review Comment: sure -- 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] ggershinsky commented on a diff in pull request #968: PARQUET-2149: Async IO implementation for ParquetFileReader
ggershinsky commented on code in PR #968: URL: https://github.com/apache/parquet-mr/pull/968#discussion_r90337 ## parquet-common/src/main/java/org/apache/parquet/bytes/AsyncMultiBufferInputStream.java: ## @@ -0,0 +1,158 @@ +/* + * 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 { +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); +LOG.debug("ASYNC (next): {}: Time blocked for read {} ns", this, timeSpent); Review Comment: should `if (LOG.isDebugEnabled()) {` be added here and in 118? This check is performed in the constructor (line 58); `nextBuffer()` is called with higher(/same) frequency. -- 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, pleas