pvary opened a new pull request, #14528:
URL: https://github.com/apache/iceberg/pull/14528
The `EncryptedOutputFile` objects generated by the
`StandardEncryptionManager.encrypt` method hide the underlying `OutputFile`.
Unfortunately, there are some hidden requirements in the Parquet
implementation for encryption to work properly.
Parquet encryption was only functioning when the target was a
`HadoopOutputFile`. Using `StandardEncryptedOutputFile.encryptingOutputFile()`
produced an `AesGcmOutputFile`, which resulted in corrupt files.
Since Parquet only needed the Hadoop Configuration object from
`HadoopOutputFile`, I addressed this by introducing:
- **HasConfiguration** interface – to expose the Hadoop configuration
(similar to `HadoopConfigurable`).
- **HadoopAesGcmOutputFile** – which extends `AesGcmOutputFile`, but also
stores the Hadoop configuration.
With these new classes, I made the following changes:
- Implemented `HasConfiguration` in both `HadoopAesGcmOutputFile` and
`HadoopOutputFile`.
- Updated the check in the `Parquet.WriteBuilder` constructor to look for
`HasConfiguration` instead of `HadoopOutputFile` when extracting the
configuration.
- Modified `ParquetIO.file` to check for `HasConfiguration` instead of
`HadoopOutputFile` when creating the `parquet.HadoopOutputFile`. - This one
seems a bit risky for me, I would like to hear your thoughts
Added a test to highlight the issue.
The test currently fails without the patch, but successful if we change:
```
Parquet.write(encryptedOutputFile.encryptingOutputFile())
```
to
```
Parquet.write(encryptedOutputFile)
```
After the patch both `OutputFile` objects could be used.
--
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]
---------------------------------------------------------------------
To unsubscribe, e-mail: [email protected]
For additional commands, e-mail: [email protected]