adamreeve commented on PR #44990:
URL: https://github.com/apache/arrow/pull/44990#issuecomment-2537316612
I tested concurrent scans of a larger dataset with uniform encryption, and
this change doesn't completely fix that scenario:
```diff
--- a/cpp/src/arrow/dataset/file_parquet_encryption_test.cc
+++ b/cpp/src/arrow/dataset/file_parquet_encryption_test.cc
@@ -289,6 +289,8 @@ TEST_F(DatasetEncryptionTest, ReadSingleFile) {
// processing encrypted datasets over 2^15 rows in multi-threaded mode.
class LargeRowEncryptionTest : public DatasetEncryptionTestBase {
public:
+ LargeRowEncryptionTest() : DatasetEncryptionTestBase(true) {}
+
// The dataset is partitioned using a Hive partitioning scheme.
void PrepareTableAndPartitioning() override {
// Specifically chosen to be greater than batch size for triggering
prefetch.
@@ -307,7 +309,7 @@ class LargeRowEncryptionTest : public
DatasetEncryptionTestBase {
// Test for writing and reading encrypted dataset with large row count.
TEST_F(LargeRowEncryptionTest, ReadEncryptLargeRows) {
- ASSERT_NO_FATAL_FAILURE(TestScanDataset());
+ ASSERT_NO_FATAL_FAILURE(TestScanDataset(true));
}
} // namespace dataset
```
This gives errors like:
```
failed with IOError: Failed decryption finalization
```
I believe this is due to the decryptor AAD being mutable. It's [updated for
each data page
read](https://github.com/apache/arrow/blob/6252e9ceeb0f8544c14f79d895a37ac198131f88/cpp/src/parquet/column_reader.cc#L357),
so concurrent use will result in incorrect AADs being used in some threads.
Rather than mutating the decryptor state, it might be possible to refactor
this so that the AAD values are passed in to the decrypt method as a parameter.
I think this should probably be addressed as a separate PR though as the
changes should be orthogonal.
--
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: [email protected]
For queries about this service, please contact Infrastructure at:
[email protected]