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

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


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

ASF GitHub Bot commented on PARQUET-2212:
-

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


##
parquet-hadoop/src/main/java/org/apache/parquet/ParquetReadOptions.java:
##
@@ -44,6 +46,9 @@ public class ParquetReadOptions {
   private static final int ALLOCATION_SIZE_DEFAULT = 8388608; // 8MB
   private static final boolean PAGE_VERIFY_CHECKSUM_ENABLED_DEFAULT = false;
   private static final boolean BLOOM_FILTER_ENABLED_DEFAULT = true;
+  // Default to true if JDK 17 or newer.
+  private static final boolean USE_OFF_HEAP_DECRYPT_BUFFER_DEFAULT =
+SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_17);

Review Comment:
   Done





> 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 a diff in pull request #1008: PARQUET-2212: Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2022-11-15 Thread GitBox


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


##
parquet-hadoop/src/main/java/org/apache/parquet/ParquetReadOptions.java:
##
@@ -44,6 +46,9 @@ public class ParquetReadOptions {
   private static final int ALLOCATION_SIZE_DEFAULT = 8388608; // 8MB
   private static final boolean PAGE_VERIFY_CHECKSUM_ENABLED_DEFAULT = false;
   private static final boolean BLOOM_FILTER_ENABLED_DEFAULT = true;
+  // Default to true if JDK 17 or newer.
+  private static final boolean USE_OFF_HEAP_DECRYPT_BUFFER_DEFAULT =
+SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_17);

Review Comment:
   Done



-- 
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-2213) Add an alternative InputFile.newStream that allow an input range

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


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

ASF GitHub Bot commented on PARQUET-2213:
-

sunchao commented on PR #1010:
URL: https://github.com/apache/parquet-mr/pull/1010#issuecomment-1315616637

   Thanks @steveloughran . I like your idea of introducing a builder api, 
although we need to think how it can be used by different implementations of 
`InputFile`, not only for Hadoop.
   
   For instance Iceberg has this 
[`S3InputFile`](https://github.com/apache/iceberg/blob/master/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java)
 which knows nothing about file status or read policy.




> Add an alternative InputFile.newStream that allow an input range
> 
>
> Key: PARQUET-2213
> URL: https://issues.apache.org/jira/browse/PARQUET-2213
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Chao Sun
>Priority: Minor
>




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


[GitHub] [parquet-mr] sunchao commented on pull request #1010: PARQUET-2213: add InputFile.newStream with a read range

2022-11-15 Thread GitBox


sunchao commented on PR #1010:
URL: https://github.com/apache/parquet-mr/pull/1010#issuecomment-1315616637

   Thanks @steveloughran . I like your idea of introducing a builder api, 
although we need to think how it can be used by different implementations of 
`InputFile`, not only for Hadoop.
   
   For instance Iceberg has this 
[`S3InputFile`](https://github.com/apache/iceberg/blob/master/aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java)
 which knows nothing about file status or read policy.


-- 
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] [Resolved] (PARQUET-2206) Microbenchmark for ColumnReadaer ReadBatch and Skip

2022-11-15 Thread Antoine Pitrou (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Antoine Pitrou resolved PARQUET-2206.
-
Fix Version/s: cpp-11.0.0
   Resolution: Fixed

Issue resolved by pull request 14523
[https://github.com/apache/arrow/pull/14523]

> Microbenchmark for ColumnReadaer ReadBatch and Skip
> ---
>
> Key: PARQUET-2206
> URL: https://issues.apache.org/jira/browse/PARQUET-2206
> Project: Parquet
>  Issue Type: Improvement
>Reporter: fatemah
>Priority: Minor
>  Labels: pull-request-available
> Fix For: cpp-11.0.0
>
>  Time Spent: 5h 50m
>  Remaining Estimate: 0h
>
>  Adding a micro benchmark for column reader ReadBatch and Skip. Later, I will 
> add benchmarks for RecordReader's ReadRecords and SkipRecords.



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


[jira] [Updated] (PARQUET-2206) Microbenchmark for ColumnReadaer ReadBatch and Skip

2022-11-15 Thread Antoine Pitrou (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Antoine Pitrou updated PARQUET-2206:

Component/s: parquet-cpp

> Microbenchmark for ColumnReadaer ReadBatch and Skip
> ---
>
> Key: PARQUET-2206
> URL: https://issues.apache.org/jira/browse/PARQUET-2206
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-cpp
>Reporter: fatemah
>Priority: Minor
>  Labels: pull-request-available
> Fix For: cpp-11.0.0
>
>  Time Spent: 5h 50m
>  Remaining Estimate: 0h
>
>  Adding a micro benchmark for column reader ReadBatch and Skip. Later, I will 
> add benchmarks for RecordReader's ReadRecords and SkipRecords.



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


[jira] [Assigned] (PARQUET-2206) Microbenchmark for ColumnReadaer ReadBatch and Skip

2022-11-15 Thread Antoine Pitrou (Jira)


 [ 
https://issues.apache.org/jira/browse/PARQUET-2206?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Antoine Pitrou reassigned PARQUET-2206:
---

Assignee: fatemah

> Microbenchmark for ColumnReadaer ReadBatch and Skip
> ---
>
> Key: PARQUET-2206
> URL: https://issues.apache.org/jira/browse/PARQUET-2206
> Project: Parquet
>  Issue Type: Improvement
>  Components: parquet-cpp
>Reporter: fatemah
>Assignee: fatemah
>Priority: Minor
>  Labels: pull-request-available
> Fix For: cpp-11.0.0
>
>  Time Spent: 5h 50m
>  Remaining Estimate: 0h
>
>  Adding a micro benchmark for column reader ReadBatch and Skip. Later, I will 
> add benchmarks for RecordReader's ReadRecords and SkipRecords.



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


[jira] [Commented] (PARQUET-2213) Add an alternative InputFile.newStream that allow an input range

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


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

ASF GitHub Bot commented on PARQUET-2213:
-

steveloughran commented on code in PR #1010:
URL: https://github.com/apache/parquet-mr/pull/1010#discussion_r1022697746


##
parquet-common/src/main/java/org/apache/parquet/io/InputFile.java:
##
@@ -41,4 +41,16 @@ public interface InputFile {
*/
   SeekableInputStream newStream() throws IOException;
 
+  /**
+   * Open a new {@link SeekableInputStream} for the underlying data file,
+   * in the range of '[offset, offset + length)'
+   *
+   * @param offset the offset in the file to read from
+   * @param length the total number of bytes to read
+   * @return a new {@link SeekableInputStream} to read the file
+   * @throws IOException if the stream cannot be opened
+   */
+  default SeekableInputStream newStream(long offset, long length) throws 
IOException {

Review Comment:
   you should go with the hadoop 
https://issues.apache.org/jira/browse/HADOOP-16202 options; s3a fs now reads 
them and it lines up abfs/gcs for the same. you can declare split start/end as 
well as file length so that
   * length => client can skip existence probes, they know the file limit
   * spilt range: they know to not prefetch past the end of the split, if they 
prefetch
   * read policy: standard set of policies and a parse policy of "csv list of 
policies -pick the first one you recognise". again, can be used by all the 
stores





> Add an alternative InputFile.newStream that allow an input range
> 
>
> Key: PARQUET-2213
> URL: https://issues.apache.org/jira/browse/PARQUET-2213
> Project: Parquet
>  Issue Type: Improvement
>Reporter: Chao Sun
>Priority: Minor
>




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


[GitHub] [parquet-mr] steveloughran commented on a diff in pull request #1010: PARQUET-2213: add InputFile.newStream with a read range

2022-11-15 Thread GitBox


steveloughran commented on code in PR #1010:
URL: https://github.com/apache/parquet-mr/pull/1010#discussion_r1022697746


##
parquet-common/src/main/java/org/apache/parquet/io/InputFile.java:
##
@@ -41,4 +41,16 @@ public interface InputFile {
*/
   SeekableInputStream newStream() throws IOException;
 
+  /**
+   * Open a new {@link SeekableInputStream} for the underlying data file,
+   * in the range of '[offset, offset + length)'
+   *
+   * @param offset the offset in the file to read from
+   * @param length the total number of bytes to read
+   * @return a new {@link SeekableInputStream} to read the file
+   * @throws IOException if the stream cannot be opened
+   */
+  default SeekableInputStream newStream(long offset, long length) throws 
IOException {

Review Comment:
   you should go with the hadoop 
https://issues.apache.org/jira/browse/HADOOP-16202 options; s3a fs now reads 
them and it lines up abfs/gcs for the same. you can declare split start/end as 
well as file length so that
   * length => client can skip existence probes, they know the file limit
   * spilt range: they know to not prefetch past the end of the split, if they 
prefetch
   * read policy: standard set of policies and a parse policy of "csv list of 
policies -pick the first one you recognise". again, can be used by all the 
stores



-- 
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-632) Parquet file in invalid state while writing to S3 from EMR

2022-11-15 Thread Emil Kleszcz (Jira)


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

Emil Kleszcz commented on PARQUET-632:
--

The same issue was observed in spark 3.2.1 when writing a parquet file to HDFS 
using parquet v. 1.12.2. 

Some of the logs from the server side in HDFS, NN:
{code:java}
<2022-11-14T08:12:31.941+0100>  : 

<2022-11-14T08:13:43.029+0100>  : 

<2022-11-14T08:14:51.025+0100>  : {code}

DN logs for one of the file blocks:
 
{code:java}
<2022-11-14T08:12:31.945+0100>  
:  <2022-11-14T08:13:43.028+0100>  
:  <2022-11-14T08:13:43.028+0100>  
:  <2022-11-14T08:15:01.095+0100>  
:
   
:
 {code}

> Parquet file in invalid state while writing to S3 from EMR
> --
>
> Key: PARQUET-632
> URL: https://issues.apache.org/jira/browse/PARQUET-632
> Project: Parquet
>  Issue Type: Bug
>Affects Versions: 1.7.0
>Reporter: Peter Halliday
>Priority: Blocker
>
> I'm writing parquet to S3 from Spark 1.6.1 on EMR.  And when it got to the 
> last few files to write to S3, I received this stacktrace in the log with no 
> other errors before or after it.  It's very consistent.  This particular 
> batch keeps erroring the same way.
> {noformat}
> 2016-06-10 01:46:05,282] WARN org.apache.spark.scheduler.TaskSetManager 
> [task-result-getter-2hread] - Lost task 3737.0 in stage 2.0 (TID 10585, 
> ip-172-16-96-32.ec2.internal): org.apache.spark.SparkException: Task failed 
> while writing rows.
>   at 
> org.apache.spark.sql.execution.datasources.DynamicPartitionWriterContainer.writeRows(WriterContainer.scala:414)
>   at 
> org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150)
>   at 
> org.apache.spark.sql.execution.datasources.InsertIntoHadoopFsRelation$$anonfun$run$1$$anonfun$apply$mcV$sp$3.apply(InsertIntoHadoopFsRelation.scala:150)
>   at org.apache.spark.scheduler.ResultTask.runTask(ResultTask.scala:66)
>   at org.apache.spark.scheduler.Task.run(Task.scala:89)
>   at org.apache.spark.executor.Executor$TaskRunner.run(Executor.scala:214)
>   at 
> java.util.concurrent.ThreadPoolExecutor.runWorker(ThreadPoolExecutor.java:1145)
>   at 
> java.util.concurrent.ThreadPoolExecutor$Worker.run(ThreadPoolExecutor.java:615)
>   at java.lang.Thread.run(Thread.java:745)
> Caused by: java.io.IOException: The file being written is in an invalid 
> state. Probably caused by an error thrown previously. Current state: COLUMN
>   at 
> org.apache.parquet.hadoop.ParquetFileWriter$STATE.error(ParquetFileWriter.java:146)
>   at 
> org.apache.parquet.hadoop.ParquetFileWriter$STATE.startBlock(ParquetFileWriter.java:138)
>   at 
> org.apache.parquet.hadoop.ParquetFileWriter.startBlock(ParquetFileWriter.java:195)
>   at 
> org.apache.parquet.hadoop.InternalParquetRecordWriter.flushRowGroupToStore(InternalParquetRecordWriter.java:153)
>   at 
> org.apache.parquet.hadoop.InternalParquetRecordWriter.close(InternalParquetRecordWriter.java:113)
>   at 
> org.apache.parquet.hadoop.ParquetRecordWriter.close(ParquetRecordWriter.java:112)
>   at 
> org.apache.spark.sql.execution.datasources.parquet.ParquetOutputWriter.close(ParquetRelation.scala:101)
>   at 
> org.apache.spark.sql.execution.datasources.DynamicPartitionWriterContainer.writeRows(WriterContainer.scala:405)
>   ... 8 more
> {noformat}



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


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

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


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

ASF GitHub Bot commented on PARQUET-2212:
-

ggershinsky commented on code in PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#discussion_r1022648985


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java:
##
@@ -133,11 +135,33 @@ public DataPage readPage() {
 public DataPage visit(DataPageV1 dataPageV1) {
   try {
 BytesInput bytes = dataPageV1.getBytes();
-if (null != blockDecryptor) {
-  bytes = 
BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dataPageAAD));
+ByteBuffer byteBuffer = bytes.toByteBuffer();
+BytesInput decompressed;
+
+if (byteBuffer.isDirect() && options.useOffHeapDecryptBuffer()) {
+  if (blockDecryptor != null) {
+byteBuffer = blockDecryptor.decrypt(byteBuffer, dataPageAAD);
+  }
+  long compressedSize = byteBuffer.limit();
+
+  ByteBuffer decompressedBuffer =
+  ByteBuffer.allocateDirect(dataPageV1.getUncompressedSize());
+  decompressor.decompress(byteBuffer, (int) compressedSize, 
decompressedBuffer,

Review Comment:
   SGTM





> 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 a diff in pull request #1008: PARQUET-2212: Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2022-11-15 Thread GitBox


ggershinsky commented on code in PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#discussion_r1022648985


##
parquet-hadoop/src/main/java/org/apache/parquet/hadoop/ColumnChunkPageReadStore.java:
##
@@ -133,11 +135,33 @@ public DataPage readPage() {
 public DataPage visit(DataPageV1 dataPageV1) {
   try {
 BytesInput bytes = dataPageV1.getBytes();
-if (null != blockDecryptor) {
-  bytes = 
BytesInput.from(blockDecryptor.decrypt(bytes.toByteArray(), dataPageAAD));
+ByteBuffer byteBuffer = bytes.toByteBuffer();
+BytesInput decompressed;
+
+if (byteBuffer.isDirect() && options.useOffHeapDecryptBuffer()) {
+  if (blockDecryptor != null) {
+byteBuffer = blockDecryptor.decrypt(byteBuffer, dataPageAAD);
+  }
+  long compressedSize = byteBuffer.limit();
+
+  ByteBuffer decompressedBuffer =
+  ByteBuffer.allocateDirect(dataPageV1.getUncompressedSize());
+  decompressor.decompress(byteBuffer, (int) compressedSize, 
decompressedBuffer,

Review Comment:
   SGTM



-- 
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-15 Thread ASF GitHub Bot (Jira)


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

ASF GitHub Bot commented on PARQUET-2212:
-

ggershinsky commented on code in PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#discussion_r1022637484


##
parquet-hadoop/src/main/java/org/apache/parquet/ParquetReadOptions.java:
##
@@ -44,6 +46,9 @@ public class ParquetReadOptions {
   private static final int ALLOCATION_SIZE_DEFAULT = 8388608; // 8MB
   private static final boolean PAGE_VERIFY_CHECKSUM_ENABLED_DEFAULT = false;
   private static final boolean BLOOM_FILTER_ENABLED_DEFAULT = true;
+  // Default to true if JDK 17 or newer.
+  private static final boolean USE_OFF_HEAP_DECRYPT_BUFFER_DEFAULT =
+SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_17);

Review Comment:
   Yep, lets do it. Given the current state of byte buffer decryption in Java 
(not all problems are resolved in JVM17), it'd be safer to keep this off, and 
let the users set to on explicitly.





> 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 a diff in pull request #1008: PARQUET-2212: Add ByteBuffer api for decryptors to allow direct memory to be decrypted

2022-11-15 Thread GitBox


ggershinsky commented on code in PR #1008:
URL: https://github.com/apache/parquet-mr/pull/1008#discussion_r1022637484


##
parquet-hadoop/src/main/java/org/apache/parquet/ParquetReadOptions.java:
##
@@ -44,6 +46,9 @@ public class ParquetReadOptions {
   private static final int ALLOCATION_SIZE_DEFAULT = 8388608; // 8MB
   private static final boolean PAGE_VERIFY_CHECKSUM_ENABLED_DEFAULT = false;
   private static final boolean BLOOM_FILTER_ENABLED_DEFAULT = true;
+  // Default to true if JDK 17 or newer.
+  private static final boolean USE_OFF_HEAP_DECRYPT_BUFFER_DEFAULT =
+SystemUtils.isJavaVersionAtLeast(JavaVersion.JAVA_17);

Review Comment:
   Yep, lets do it. Given the current state of byte buffer decryption in Java 
(not all problems are resolved in JVM17), it'd be safer to keep this off, and 
let the users set to on explicitly.



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