rdblue commented on code in PR #9752:
URL: https://github.com/apache/iceberg/pull/9752#discussion_r1501944749
##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -20,39 +20,33 @@
import org.apache.iceberg.io.InputFile;
import org.apache.iceberg.io.SeekableInputStream;
-import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
public class AesGcmInputFile implements InputFile {
private final InputFile sourceFile;
private final byte[] dataKey;
private final byte[] fileAADPrefix;
private long plaintextLength;
+ /**
+ * Important: sourceFile.getLength() must return the verified plaintext
content length, not the
+ * physical file size after encryption. This protects against tampering with
the file size in
+ * untrusted storage systems.
+ */
public AesGcmInputFile(InputFile sourceFile, byte[] dataKey, byte[]
fileAADPrefix) {
this.sourceFile = sourceFile;
this.dataKey = dataKey;
this.fileAADPrefix = fileAADPrefix;
- this.plaintextLength = -1;
+ this.plaintextLength = sourceFile.getLength();
Review Comment:
Another problem I just ran into is that if we store the plaintext length,
then `EncryptingFileIO` doesn't know what length to pass into
`FileIO.newInputFile(String, long)` for the encrypted file length. Encrypted
length depends on the encryption scheme and we don't know the details of the
scheme used by some `EncryptionManager`.
That makes me think that we should store the encrypted length in Iceberg
metadata. That is also intuitive because the metadata length always matches the
object storage length.
We'll have to find out how to get the encrypted length in the data and
manifest write paths.
--
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]