[jira] [Commented] (PARQUET-2149) Implement async IO for Parquet file reader

2022-11-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17635582#comment-17635582
 ] 

ASF GitHub Bot commented on PARQUET-2149:
-

parthchandra commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1319411064

   > > IMO, switching `ioThreadPool` and `processThreadPool` the reader 
instance level will make it more flexible.
   > 
   > I've changed the thread pool so that it is not initialized by default but 
I left them as static members. Ideally, there should be a single IO thread pool 
that handles all the IO for a process and the size of the pool is determined by 
the bandwidthof the underlying storage system. Making them per instance is not 
an issue though. The calling code can decide to set the same thread pool for 
all instances and achieve the same result. Let me update this.
   > 
   > Also, any changes you want to make are fine with me, and the help is 
certainly appreciated !
   
   I'm thinking of merging the thread pools into a single ioThreadPool and 
making the ioThreadPool settable thru `ParquetReadOptions` (like the allocator 
is). The work being done by the processThreadPool is rather small and maybe we 
can do away with it. 
   Adding the pool via `ParquetReadOptions`  makes it easier to use with 
`ParquetReader` (used a lot in unit tests).
   WDYT?




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


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

2022-11-17 Thread GitBox


parthchandra commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1319411064

   > > IMO, switching `ioThreadPool` and `processThreadPool` the reader 
instance level will make it more flexible.
   > 
   > I've changed the thread pool so that it is not initialized by default but 
I left them as static members. Ideally, there should be a single IO thread pool 
that handles all the IO for a process and the size of the pool is determined by 
the bandwidthof the underlying storage system. Making them per instance is not 
an issue though. The calling code can decide to set the same thread pool for 
all instances and achieve the same result. Let me update this.
   > 
   > Also, any changes you want to make are fine with me, and the help is 
certainly appreciated !
   
   I'm thinking of merging the thread pools into a single ioThreadPool and 
making the ioThreadPool settable thru `ParquetReadOptions` (like the allocator 
is). The work being done by the processThreadPool is rather small and maybe we 
can do away with it. 
   Adding the pool via `ParquetReadOptions`  makes it easier to use with 
`ParquetReader` (used a lot in unit tests).
   WDYT?


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



[jira] [Commented] (PARQUET-2212) Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2022-11-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17635581#comment-17635581
 ] 

ASF GitHub Bot commented on PARQUET-2212:
-

parthchandra commented on PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#issuecomment-1319405608

   Thank you @ggershinsky  !




> Add ByteBuffer api for decryptors to allow direct memory to be decrypted
> 
>
> Key: PARQUET-2212
> URL: https://issues.apache.org/jira/browse/PARQUET-2212
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.3
>Reporter: Parth Chandra
>Priority: Major
> Fix For: 1.12.3
>
>
> The decrypt API in BlockCipher.Decryptor currently only provides an api that 
> takes in a byte array
> {code:java}
> byte[] decrypt(byte[] lengthAndCiphertext, byte[] AAD);{code}
> A parquet reader that uses the DirectByteBufferAllocator has to incur the 
> cost of copying the data into a byte array (and sometimes back to a 
> DirectByteBuffer) to decrypt data.
> This proposes adding a new API that accepts ByteBuffer as input and avoids 
> the data copy.
> {code:java}
> ByteBuffer decrypt(ByteBuffer from, byte[] AAD);{code}
> The decryption in ColumnChunkPageReadStore can also be updated to use the 
> ByteBuffer based api if the buffer is a DirectByteBuffer. If the buffer is a 
> HeapByteBuffer, then we can continue to use the byte array API since that 
> does not incur a copy when the underlying byte array is accessed.
> Also, some investigation has shown that decryption with ByteBuffers is not 
> able to use hardware acceleration in JVM's before JDK17. In those cases, the 
> overall decryption speed is faster with byte arrays even after incurring the 
> overhead of making a copy. 
> The proposal, then, is to enable the use of the ByteBuffer api for 
> DirectByteBuffers only, and only if the JDK is JDK17 or higher or the user 
> explicitly configures it. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] parthchandra commented on pull request #1008: PARQUET-2212: Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2022-11-17 Thread GitBox


parthchandra commented on PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#issuecomment-1319405608

   Thank you @ggershinsky  !


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



[jira] [Commented] (PARQUET-2149) Implement async IO for Parquet file reader

2022-11-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17635497#comment-17635497
 ] 

ASF GitHub Bot commented on PARQUET-2149:
-

parthchandra commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1319040697

   @wgtmac thank you for looking at this. I don't have any more TODOs on this 
PR. 
   
   > * Adopt the incoming Hadoop vectored io api.
   
   This should be part of another PR. There is a draft PR (#999 ) open for 
this. Once that is merged in, I can revisit the async I/O code and incorporate 
the vectored io api. 
   In other experiments I have seen that async io gives better results over 
slower networks. With faster network connections, as  is the case where we are 
reading from S3 within an AWS environment, reading in parallel (as the vector 
io api does), starts to give better results. 
   I believe, that both should be available as options. 
   
   > * Benchmark against remote object stores from different cloud providers.
   
   The numbers I posted earlier were for reading from AWS/S3 over a 1 Gbps 
line. Reading from within AWS shows lesser improvement. I don't have an account 
with other cloud providers. Any help here would be appreciated. 
   
   > IMO, switching `ioThreadPool` and `processThreadPool` the reader instance 
level will make it more flexible.
   
   I've changed the thread pool so that it is not initialized by default but I 
left them as static members. Ideally, there should be a single IO thread pool 
that handles all the IO for a process and the size of the pool is determined by 
the bandwidthof the underlying storage system. 
   Making them per instance is not an issue though. The calling code can decide 
to set the same thread pool for all instances and achieve the same result. 
   Let me update this. 
   
   Also, any changes you want to make are fine with me, and the help is 
certainly appreciated !
   




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


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

2022-11-17 Thread GitBox


parthchandra commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1319040697

   @wgtmac thank you for looking at this. I don't have any more TODOs on this 
PR. 
   
   > * Adopt the incoming Hadoop vectored io api.
   
   This should be part of another PR. There is a draft PR (#999 ) open for 
this. Once that is merged in, I can revisit the async I/O code and incorporate 
the vectored io api. 
   In other experiments I have seen that async io gives better results over 
slower networks. With faster network connections, as  is the case where we are 
reading from S3 within an AWS environment, reading in parallel (as the vector 
io api does), starts to give better results. 
   I believe, that both should be available as options. 
   
   > * Benchmark against remote object stores from different cloud providers.
   
   The numbers I posted earlier were for reading from AWS/S3 over a 1 Gbps 
line. Reading from within AWS shows lesser improvement. I don't have an account 
with other cloud providers. Any help here would be appreciated. 
   
   > IMO, switching `ioThreadPool` and `processThreadPool` the reader instance 
level will make it more flexible.
   
   I've changed the thread pool so that it is not initialized by default but I 
left them as static members. Ideally, there should be a single IO thread pool 
that handles all the IO for a process and the size of the pool is determined by 
the bandwidthof the underlying storage system. 
   Making them per instance is not an issue though. The calling code can decide 
to set the same thread pool for all instances and achieve the same result. 
   Let me update this. 
   
   Also, any changes you want to make are fine with me, and the help is 
certainly appreciated !
   


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



[jira] [Commented] (PARQUET-2149) Implement async IO for Parquet file reader

2022-11-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2149?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17635445#comment-17635445
 ] 

ASF GitHub Bot commented on PARQUET-2149:
-

wgtmac commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1318904541

   It looks like this PR is complete and all comments are addressed except some 
outstanding ones:
   
   - Adopt the incoming Hadoop vectored io api.
   - Benchmark against remote object stores from different cloud providers.
   
   IMO, switching `ioThreadPool` and `processThreadPool` the reader instance 
level will make it more flexible. 
   
   @parthchandra Do you have any TODO list for this PR? If you are busy with 
some other stuff, I can continue to work on it because it is really a nice 
improvement.
   
   cc @shangxinli 




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


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

2022-11-17 Thread GitBox


wgtmac commented on PR #968:
URL: https://github.com/apache/parquet-mr/pull/968#issuecomment-1318904541

   It looks like this PR is complete and all comments are addressed except some 
outstanding ones:
   
   - Adopt the incoming Hadoop vectored io api.
   - Benchmark against remote object stores from different cloud providers.
   
   IMO, switching `ioThreadPool` and `processThreadPool` the reader instance 
level will make it more flexible. 
   
   @parthchandra Do you have any TODO list for this PR? If you are busy with 
some other stuff, I can continue to work on it because it is really a nice 
improvement.
   
   cc @shangxinli 


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



[jira] [Commented] (PARQUET-2212) Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2022-11-17 Thread ASF GitHub Bot (Jira)


[ 
https://issues.apache.org/jira/browse/PARQUET-2212?page=com.atlassian.jira.plugin.system.issuetabpanels:comment-tabpanel&focusedCommentId=17635350#comment-17635350
 ] 

ASF GitHub Bot commented on PARQUET-2212:
-

ggershinsky commented on PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#issuecomment-1318625808

   Thank you. This PR is significant enough to be reviewed by more than person. 
Cc @shangxinli 




> Add ByteBuffer api for decryptors to allow direct memory to be decrypted
> 
>
> Key: PARQUET-2212
> URL: https://issues.apache.org/jira/browse/PARQUET-2212
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-mr
>Affects Versions: 1.12.3
>Reporter: Parth Chandra
>Priority: Major
> Fix For: 1.12.3
>
>
> The decrypt API in BlockCipher.Decryptor currently only provides an api that 
> takes in a byte array
> {code:java}
> byte[] decrypt(byte[] lengthAndCiphertext, byte[] AAD);{code}
> A parquet reader that uses the DirectByteBufferAllocator has to incur the 
> cost of copying the data into a byte array (and sometimes back to a 
> DirectByteBuffer) to decrypt data.
> This proposes adding a new API that accepts ByteBuffer as input and avoids 
> the data copy.
> {code:java}
> ByteBuffer decrypt(ByteBuffer from, byte[] AAD);{code}
> The decryption in ColumnChunkPageReadStore can also be updated to use the 
> ByteBuffer based api if the buffer is a DirectByteBuffer. If the buffer is a 
> HeapByteBuffer, then we can continue to use the byte array API since that 
> does not incur a copy when the underlying byte array is accessed.
> Also, some investigation has shown that decryption with ByteBuffers is not 
> able to use hardware acceleration in JVM's before JDK17. In those cases, the 
> overall decryption speed is faster with byte arrays even after incurring the 
> overhead of making a copy. 
> The proposal, then, is to enable the use of the ByteBuffer api for 
> DirectByteBuffers only, and only if the JDK is JDK17 or higher or the user 
> explicitly configures it. 



--
This message was sent by Atlassian Jira
(v8.20.10#820010)


[GitHub] [parquet-mr] ggershinsky commented on pull request #1008: PARQUET-2212: Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2022-11-17 Thread GitBox


ggershinsky commented on PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#issuecomment-1318625808

   Thank you. This PR is significant enough to be reviewed by more than person. 
Cc @shangxinli 


-- 
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] Jimexist opened a new pull request, #1012: Update README.md to reflect thrift 0.17

2022-11-17 Thread GitBox


Jimexist opened a new pull request, #1012:
URL: https://github.com/apache/parquet-mr/pull/1012

   now that we have 0.17 to use
   
   Make sure you have checked _all_ steps below.
   
   ### Jira
   
   - [ ] My PR addresses the following [Parquet 
Jira](https://issues.apache.org/jira/browse/PARQUET/) issues and references 
them in the PR title. For example, "PARQUET-1234: My Parquet PR"
 - https://issues.apache.org/jira/browse/PARQUET-XXX
 - In case you are adding a dependency, check if the license complies with 
the [ASF 3rd Party License 
Policy](https://www.apache.org/legal/resolved.html#category-x).
   
   ### Tests
   
   - [ ] My PR adds the following unit tests __OR__ does not need testing for 
this extremely good reason:
   
   ### Commits
   
   - [ ] My commits all reference Jira issues in their subject lines. In 
addition, my commits follow the guidelines from "[How to write a good git 
commit message](http://chris.beams.io/posts/git-commit/)":
 1. Subject is separated from body by a blank line
 1. Subject is limited to 50 characters (not including Jira issue reference)
 1. Subject does not end with a period
 1. Subject uses the imperative mood ("add", not "adding")
 1. Body wraps at 72 characters
 1. Body explains "what" and "why", not "how"
   
   ### Documentation
   
   - [ ] In case of new functionality, my PR adds documentation that describes 
how to use it.
 - All the public functions and the classes in the PR contain Javadoc that 
explain what it does
   


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