You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@iceberg.apache.org by GitBox <gi...@apache.org> on 2022/06/23 12:52:16 UTC

[GitHub] [iceberg] ggershinsky opened a new pull request, #3231: GCM encryption stream

ggershinsky opened a new pull request, #3231:
URL: https://github.com/apache/iceberg/pull/3231

   Implements #2060
   
   Besides encrypting and signing metadata (avro, json, puffin), this stream can also encrypt Avro data.
   A subset of this stream can be used for adding integrity protection for encrypted ORC data.
   
   @RussellSpitzer @jackye1995 @flyrain @rdblue


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky closed pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky closed pull request #3231: GCM encryption stream
URL: https://github.com/apache/iceberg/pull/3231


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904090625


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);

Review Comment:
   Can we make the math here a little more explicit, all of the -1 and such make things a little hard for me to read. We discussed this a bit before offline, I mainly just don't want to have to think about why we have a +1 here or why we have a -1 somewhere else etc ...
   
   I think we were discussing a something like "number of full blocks", "remainder" etc ...



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904530443


##########
core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Random;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestGcmStreams {
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Test
+  public void testRandomWriteRead() throws IOException {

Review Comment:
   Sure, will add



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1039941138


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);

Review Comment:
   Pass AAD?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278601366


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -101,37 +117,52 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad) {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(
+        byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) {
       Preconditions.checkState(
-          plainTextLength >= 1,
+          ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length "
               + ciphertext.length
               + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted"
               + " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);

Review Comment:
   The allocation here is unnecessary. `GCMParameterSpec` accepts an offset and length for the nonce array.



##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -101,37 +117,52 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad) {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(
+        byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) {
       Preconditions.checkState(
-          plainTextLength >= 1,
+          ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length "
               + ciphertext.length
               + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted"
               + " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);

Review Comment:
   The allocation and copy here is unnecessary. `GCMParameterSpec` accepts an offset and length for the nonce array.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278598513


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,116 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+  public static final int plainBlockSize = 1024 * 1024;

Review Comment:
   `static final` variables should use names that are ALL_CAPS.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292844218


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -53,29 +70,61 @@ public AesGcmEncryptor(byte[] keyBytes) {
       }
 
       this.randomGenerator = new SecureRandom();
+      this.nonce = new byte[NONCE_LENGTH];
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
-      byte[] nonce = new byte[NONCE_LENGTH];
-      randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;
       byte[] cipherText = new byte[cipherTextLength];
+      encrypt(plaintext, plaintextOffset, plaintextLength, cipherText, 0, aad);
+      return cipherText;
+    }
+
+    public int encrypt(
+        byte[] plaintext,
+        int plaintextOffset,
+        int plaintextLength,
+        byte[] ciphertextBuffer,
+        int ciphertextOffset,
+        byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);

Review Comment:
   We typically use `Invalid plain text length: %s`:
   * "Invalid" is used more often and signals that the value cannot be used, while "wrong" could indicate that the value was not equal to some expected value or failed a different type of check.
   * Variable names like `plaintextLength` aren't helpful to people reading error messages because they aren't familiar with this code. Using plain English to describe the problem is better.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1293181654


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -53,29 +70,61 @@ public AesGcmEncryptor(byte[] keyBytes) {
       }
 
       this.randomGenerator = new SecureRandom();
+      this.nonce = new byte[NONCE_LENGTH];
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
-      byte[] nonce = new byte[NONCE_LENGTH];
-      randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;
       byte[] cipherText = new byte[cipherTextLength];
+      encrypt(plaintext, plaintextOffset, plaintextLength, cipherText, 0, aad);
+      return cipherText;
+    }
+
+    public int encrypt(
+        byte[] plaintext,
+        int plaintextOffset,
+        int plaintextLength,
+        byte[] ciphertextBuffer,
+        int ciphertextOffset,
+        byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
+      randomGenerator.nextBytes(nonce);
+      int enciphered;
 
       try {
         GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
         cipher.init(Cipher.ENCRYPT_MODE, aesKey, spec);
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
+        enciphered =
+            cipher.doFinal(
+                plaintext,
+                plaintextOffset,
+                plaintextLength,
+                ciphertextBuffer,
+                ciphertextOffset + NONCE_LENGTH);

Review Comment:
   This is done outside `doFinal`, I'll add a comment.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r929260153


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -53,10 +63,14 @@ public AesGcmEncryptor(byte[] keyBytes) {
       this.randomGenerator = new SecureRandom();
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {

Review Comment:
   Where is this encryption format documented so I can take a look at compare it to the spec?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#issuecomment-1161395051

   > Thanks @ggershinsky for the patch. Can we add a unit test for these two classes?
   
   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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#issuecomment-1181747028

   SGTM. I've added a a commit with an optimization; no other changes are planned, so the PR is good to go from my POV.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902964917


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+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 InputFile sourceFile;
+  private byte[] dataKey;
+  private long plaintextLength;
+
+  public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) {
+    this.sourceFile = sourceFile;
+    this.dataKey = dataKey;
+    this.plaintextLength = -1;
+  }
+
+  @Override
+  public long getLength() {
+    Preconditions.checkArgument(plaintextLength >= 0, "Length is known after new stream is created");

Review Comment:
   Should we set the length if field plaintextLength is -1? Looks like this method is valid only after `newStream()` is called.
   ```
   if(plaintextLength == -1) {
   this.newStream();
   }
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904343968


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;

Review Comment:
   i think we could also probably call this "cypherBytesToLoad"? 



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904600077


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }
+
+      // Copy nonce
+      System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH);
+
+      byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock;
+      try {
+        GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce);
+        gcmCipher.init(Cipher.DECRYPT_MODE, key, spec);
+        gcmCipher.updateAAD(aad);
+
+        plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH,
+                toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH);
+      } catch (GeneralSecurityException e) {
+        throw new IOException("Failed to decrypt", e);
+      }
+
+      int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remaining >= remainingInBlock;
+      int toCopy = finishTheBlock ? remainingInBlock : remaining;
+
+      System.arraycopy(plaintextBlock, currentOffsetInPlainBlock, b, resultBufferOffset, toCopy);
+      remaining -= toCopy;
+      resultBufferOffset += toCopy;
+      currentOffsetInPlainBlock += toCopy;
+      boolean endOfStream = lastBlock && finishTheBlock;

Review Comment:
   remaining can be > 0 ; it just means the stream had less bytes than `len`, and the method returns ` len - remaining`



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902969750


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }
+
+      // Copy nonce
+      System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH);
+
+      byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock;
+      try {
+        GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce);
+        gcmCipher.init(Cipher.DECRYPT_MODE, key, spec);
+        gcmCipher.updateAAD(aad);
+
+        plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH,
+                toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH);
+      } catch (GeneralSecurityException e) {
+        throw new IOException("Failed to decrypt", e);
+      }
+
+      int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remaining >= remainingInBlock;
+      int toCopy = finishTheBlock ? remainingInBlock : remaining;
+
+      System.arraycopy(plaintextBlock, currentOffsetInPlainBlock, b, resultBufferOffset, toCopy);
+      remaining -= toCopy;
+      resultBufferOffset += toCopy;
+      currentOffsetInPlainBlock += toCopy;
+      boolean endOfStream = lastBlock && finishTheBlock;
+      if (endOfStream) {
+        break;
+      }
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+      }
+    }
+
+    plainStreamPosition += len - remaining;
+    return len - remaining;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos >= plainStreamSize) {
+      throw new IOException("At or beyond max stream size " + plainStreamSize + ", " + newPos);

Review Comment:
   The massage could be
   ```
   "The seeking position " + newPos + " has reached or exceeded the max stream size "  + plainStreamSize.
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278603432


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+    currentDecryptedBlock = null;
+  }
+
+  static long calculatePlaintextLength(long sourceLength, int plainBlockSize) {
+    long netSourceFileLength = netSourceFileLength(sourceLength);
+
+    if (netSourceFileLength == 0) {
+      return 0;
+    }
+
+    int cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;

Review Comment:
   I think that this should be a constant and that `plainBlockSize` should not be passed in. I realize that the purpose is to make this generic if we ever want to support another block size, but all it does right now is make the code more confusing. I'm okay keeping the block size in the header, but we should not pass it around in the code. We can always update the code later.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#issuecomment-1657846424

   > I had a lot of comments, so I went ahead and implemented most of them on top of this. I think it simplified how the input stream works quite a bit and I fixed a lot of the comments that I made. Please see https://github.com/ggershinsky/iceberg/pull/6.
   
   Thanks! Merged the PR.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1281460167


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {

Review Comment:
   I've implemented `write(int)` by calling `write(byte[])`, which in turn calls this method (`write(byte[], offset, len)`). So all of accounting and encryption is in one place.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1279977810


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;

Review Comment:
   Shouldn't this always be `targetStream.pos() + positionInBuffer`?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292844564


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -53,29 +70,61 @@ public AesGcmEncryptor(byte[] keyBytes) {
       }
 
       this.randomGenerator = new SecureRandom();
+      this.nonce = new byte[NONCE_LENGTH];
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
-      byte[] nonce = new byte[NONCE_LENGTH];
-      randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;
       byte[] cipherText = new byte[cipherTextLength];
+      encrypt(plaintext, plaintextOffset, plaintextLength, cipherText, 0, aad);
+      return cipherText;
+    }
+
+    public int encrypt(
+        byte[] plaintext,
+        int plaintextOffset,
+        int plaintextLength,
+        byte[] ciphertextBuffer,
+        int ciphertextOffset,
+        byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
+      randomGenerator.nextBytes(nonce);
+      int enciphered;
 
       try {
         GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
         cipher.init(Cipher.ENCRYPT_MODE, aesKey, spec);
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
+        enciphered =
+            cipher.doFinal(
+                plaintext,
+                plaintextOffset,
+                plaintextLength,
+                ciphertextBuffer,
+                ciphertextOffset + NONCE_LENGTH);
+
+        if (enciphered != plaintextLength + GCM_TAG_LENGTH) {
+          throw new RuntimeException(
+              "Wrong number of enciphered bytes: "
+                  + enciphered
+                  + ". Must be "
+                  + plaintextLength
+                  + GCM_TAG_LENGTH);

Review Comment:
   I think this can be more clear because it doesn't tell the reader that the encryption didn't work right.
   
   Instead, how about `Failed to encrypt block: expected %s encrypted bytes but produced %s bytes`?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044725178


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);

Review Comment:
   Nit: name should be `isLastBlock`



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#issuecomment-1195769235

   Ack, I'm on it


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902974131


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.SecureRandom;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.primitives.Ints;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+  // AES-GCM parameters
+  public static final int GCM_NONCE_LENGTH = 12; // in bytes
+  public static final int GCM_TAG_LENGTH = 16; // in bytes
+  public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+  public static final String MAGIC_STRING = "GCM1";
+
+  static final byte[] MAGIC_ARRAY = MAGIC_STRING.getBytes(StandardCharsets.UTF_8);
+  static final int PREFIX_LENGTH = MAGIC_ARRAY.length + 4; // magic_len + block_size_len
+
+  private PositionOutputStream targetStream;
+
+  private Cipher gcmCipher;
+  private SecureRandom random;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private int blockSize = 1024 * 1024;
+  private byte[] plaintextBlockBuffer;
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+  private byte[] fileAadPrefix;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.targetStream = targetStream;
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.random = new SecureRandom();
+    this.nonce = new byte[GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.plaintextBlockBuffer = new byte[blockSize];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] prefixBytes = ByteBuffer.allocate(PREFIX_LENGTH).order(ByteOrder.LITTLE_ENDIAN)
+        .put(MAGIC_ARRAY)
+        .putInt(blockSize)
+        .array();
+    targetStream.write(prefixBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b)  throws IOException {
+    write(b, 0, b.length);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = blockSize - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plaintextBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == blockSize) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {
+    random.nextBytes(nonce);
+    GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
+    try {
+      gcmCipher.init(Cipher.ENCRYPT_MODE, key, spec);
+    } catch (InvalidKeyException | InvalidAlgorithmParameterException e) {
+      throw new IOException("Failed to init GCM cipher", e);
+    }
+
+    byte[] aad = calculateAAD(fileAadPrefix, currentBlockIndex);
+    gcmCipher.updateAAD(aad);
+
+    byte[] cipherText = new byte[GCM_NONCE_LENGTH + positionInBuffer + GCM_TAG_LENGTH];
+    System.arraycopy(nonce, 0, cipherText, 0, GCM_NONCE_LENGTH);
+    try {
+      int encrypted = gcmCipher.doFinal(plaintextBlockBuffer, 0, positionInBuffer, cipherText, GCM_NONCE_LENGTH);
+      Preconditions.checkArgument((encrypted == (positionInBuffer + GCM_TAG_LENGTH)),
+          "Wrong length of encrypted output: " + encrypted + " vs " + (positionInBuffer + GCM_TAG_LENGTH));
+    } catch (GeneralSecurityException e) {
+      throw new IOException("Failed to encrypt", e);
+    }
+
+    currentBlockIndex++;
+
+    targetStream.write(cipherText);
+  }
+
+  static byte[] calculateAAD(byte[] fileAadPrefix, int currentBlockIndex) {

Review Comment:
   This can go to a common place as well since both classes need it. We may have a util class, e.g. `GcmUtil`



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902279615


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+
+import static org.apache.iceberg.encryption.AesGcmOutputStream.HEADER_SIZE_LENGTH;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,

Review Comment:
   Always created by an `AesGcmInputFile` in the same package.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904337068


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;

Review Comment:
   This is part of the code I think is kind of complicated because of the special casing of the last block logic. Let's try to make this as clear as possible. 
   
   Do we actually need to keep these numbers here? I was thinking about this again, could we instead do something like
   
   Math.min(cipherBlockSize, remaining)?



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;

Review Comment:
   This is part of the code I think is kind of complicated because of the special casing of the last block logic. Let's try to make this as clear as possible. 
   
   Do we actually need to keep these numbers here? I was thinking about this again, could we instead do something like
   ```
   Math.min(cipherBlockSize, remaining)?
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1041943081


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);

Review Comment:
   This is the last block, so its length will be variable and shorter than the block size constant.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1054083301


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(Ciphers.GCM_STREAM_PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastCipherBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Should read " + toLoad + " bytes, but got only " + loaded + " bytes");
+      }
+
+      byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock = gcmDecryptor.decrypt(ciphertextBlockBuffer, 0, toLoad, aad);

Review Comment:
   Yep, the current code is focused on manifest files, so I thought of adding this optimization to a later version for heavier  Avro data files. But I can add a single-block buffer (for the "current position block") now, shouldn't be too much extra complexity. Later, we can consider multi-block buffers and their management.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1039937969


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(Ciphers.GCM_STREAM_PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);

Review Comment:
   It would be good to have some of these calculations as util methods. This could be `blockOffset(blockIndex)`.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1279978190


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();

Review Comment:
   I think this needs to be implemented as well.



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {

Review Comment:
   Do you think it would be good to have a boolean `fullBlock` to keep track of whether the last block was a full block? Then we could check it here and refuse to write another block if the last block was not full.



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {

Review Comment:
   You'll probably want to move this to a method that can be called from `write(int)` like `flushBlock()`.



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;

Review Comment:
   While it should be the same, I think that this should use `plainBlockBuffer.length` instead of `Ciphers.PLAIN_BLOCK_SIZE` because that's the length that will avoid an `ArrayIndexOutOfBoundsException`. Otherwise, we're relying on the assumption that `plainBlockBuffer.length == Ciphers.PLAIN_BLOCK_SIZE` that may not hold later.



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;

Review Comment:
   Nit: missing newline above this.



##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -55,10 +71,15 @@ public AesGcmEncryptor(byte[] keyBytes) {
       this.randomGenerator = new SecureRandom();
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
       byte[] nonce = new byte[NONCE_LENGTH];
       randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;

Review Comment:
   Is this buffer allocation necessary? Why can't we pass this in and reuse it for every block? It is passed into the `doFinal` call so I think we could have this return the number of bytes that were encrypted and reuse a buffer from the caller.



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;

Review Comment:
   I think it's better to use `Math.min(freeBlockBytes, remaining)`.



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;

Review Comment:
   I think it would be better to do all of the offset accounting in the same place, rather than breaking it up with the check for a full block.



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {
+    if (currentBlockIndex == Integer.MAX_VALUE) {
+      throw new IOException("Too many blocks - exceed Integer.MAX_VALUE");
+    }
+
+    byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex);
+    byte[] cipherBlockBuffer = gcmEncryptor.encrypt(plainBlockBuffer, 0, positionInBuffer, aad);
+    currentBlockIndex++;
+    targetStream.write(cipherBlockBuffer);

Review Comment:
   I think that this should reset `positionInBuffer` instead of doing it in the caller.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1281423405


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final byte[] fileAADPrefix;
+  private final Ciphers.AesGcmDecryptor decryptor;
+  private final byte[] cipherBlockBuffer;
+  private final long numBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+
+  private long plainStreamPosition;
+  private long currentPlainBlockIndex;
+  private byte[] currentPlainBlock;
+  private int currentPlainBlockSize;
+  private byte[] singleByte;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix) {
+    this.sourceStream = sourceStream;
+    this.fileAADPrefix = fileAADPrefix;
+    this.decryptor = new Ciphers.AesGcmDecryptor(aesKey);
+    this.cipherBlockBuffer = new byte[Ciphers.CIPHER_BLOCK_SIZE];
+
+    this.plainStreamPosition = 0;
+    this.currentPlainBlockIndex = -1;
+    this.currentPlainBlock = null;
+    this.currentPlainBlockSize = 0;
+
+    long streamLength = sourceLength - Ciphers.GCM_STREAM_HEADER_LENGTH;
+    long numFullBlocks = Math.toIntExact(streamLength / Ciphers.CIPHER_BLOCK_SIZE);
+    long cipherFullBlockLength = numFullBlocks * Ciphers.CIPHER_BLOCK_SIZE;
+    int cipherBytesInLastBlock = Math.toIntExact(streamLength - cipherFullBlockLength);
+    boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+    this.numBlocks = fullBlocksOnly ? numFullBlocks : numFullBlocks + 1;
+    this.lastCipherBlockSize =
+        fullBlocksOnly ? Ciphers.CIPHER_BLOCK_SIZE : cipherBytesInLastBlock; // never 0
+
+    long lastPlainBlockSize =
+        (long) lastCipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH;
+    this.plainStreamSize =
+        numFullBlocks * Ciphers.PLAIN_BLOCK_SIZE + (fullBlocksOnly ? 0 : lastPlainBlockSize);
+    this.singleByte = new byte[1];
+  }
+
+  private void validateHeader() throws IOException {
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+
+    Preconditions.checkState(
+        Ciphers.GCM_STREAM_MAGIC.equals(ByteBuffer.wrap(headerBytes, 0, 4)),
+        "Invalid GCM stream: magic does not match AGS1");
+
+    int plainBlockSize = ByteBuffer.wrap(headerBytes, 4, 4).order(ByteOrder.LITTLE_ENDIAN).getInt();
+    Preconditions.checkState(
+        plainBlockSize == Ciphers.PLAIN_BLOCK_SIZE,
+        "Invalid GCM stream: block size %d != %d",
+        plainBlockSize,
+        Ciphers.PLAIN_BLOCK_SIZE);
+  }
+
+  @Override
+  public int available() {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  private int availableInCurrentBlock() {
+    if (currentPlainBlockIndex < 0) {

Review Comment:
   SGTM. Applied this change.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292843270


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] fileAadPrefix;
+  private final byte[] singleByte;
+
+  private byte[] plainBlock;
+  private byte[] cipherBlock;
+  private int positionInPlainBlock;
+  private long streamPosition;
+  private int currentBlockIndex;
+  private boolean isHeaderWritten;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlock = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.cipherBlock = new byte[Ciphers.CIPHER_BLOCK_SIZE];
+    this.positionInPlainBlock = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+    this.isHeaderWritten = false;
+    this.singleByte = new byte[1];
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    singleByte[0] = (byte) (b & 0x000000FF);
+    write(singleByte);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (!isHeaderWritten) {
+      writeHeader();
+    }
+
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = plainBlock.length - positionInPlainBlock;
+      int toWrite = Math.min(freeBlockBytes, remaining);
+
+      System.arraycopy(b, offset, plainBlock, positionInPlainBlock, toWrite);
+      positionInPlainBlock += toWrite;
+      offset += toWrite;
+      remaining -= toWrite;
+
+      if (positionInPlainBlock == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+      }
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!isHeaderWritten) {
+      writeHeader();
+    }
+
+    if (positionInPlainBlock > 0) {
+      encryptAndWriteBlock();
+    }
+
+    targetStream.close();
+    plainBlock = null;
+    cipherBlock = null;
+  }
+
+  private void writeHeader() throws IOException {
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+    isHeaderWritten = true;
+  }
+
+  private void encryptAndWriteBlock() throws IOException {
+    if (currentBlockIndex == Integer.MAX_VALUE) {
+      throw new IOException("Too many blocks - exceed Integer.MAX_VALUE");

Review Comment:
   I think this should be `Cannot write block: exceeded Integer.MAX_VALUE blocks`.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278602843


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+    currentDecryptedBlock = null;
+  }
+
+  static long calculatePlaintextLength(long sourceLength, int plainBlockSize) {
+    long netSourceFileLength = netSourceFileLength(sourceLength);

Review Comment:
   As I noted below, I don't think "net" is very clear. How about `long streamLength = sourceLength - Ciphers.GCM_STREAM_HEADER_LENGTH`?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278601269


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -67,7 +83,7 @@ public byte[] encrypt(byte[] plainText, byte[] aad) {
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
+        cipher.doFinal(plaintext, plaintextOffset, plaintextLength, cipherText, NONCE_LENGTH);

Review Comment:
   Can you confirm that `doFinal` also writes the GCM tag to the output buffer?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902277365


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+
+import static org.apache.iceberg.encryption.AesGcmOutputStream.HEADER_SIZE_LENGTH;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;

Review Comment:
   sure



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902277365


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+
+import static org.apache.iceberg.encryption.AesGcmOutputStream.HEADER_SIZE_LENGTH;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;

Review Comment:
   sure, will change in the next commit



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902977462


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);

Review Comment:
   Add message for easier debugging?
   ```
   new IOException("Failed to get an instance of GCM cipher",  e)
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904351686


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }
+
+      // Copy nonce
+      System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH);
+
+      byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock;
+      try {
+        GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce);
+        gcmCipher.init(Cipher.DECRYPT_MODE, key, spec);
+        gcmCipher.updateAAD(aad);
+
+        plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH,
+                toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH);
+      } catch (GeneralSecurityException e) {
+        throw new IOException("Failed to decrypt", e);
+      }
+
+      int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remaining >= remainingInBlock;
+      int toCopy = finishTheBlock ? remainingInBlock : remaining;
+
+      System.arraycopy(plaintextBlock, currentOffsetInPlainBlock, b, resultBufferOffset, toCopy);
+      remaining -= toCopy;
+      resultBufferOffset += toCopy;
+      currentOffsetInPlainBlock += toCopy;
+      boolean endOfStream = lastBlock && finishTheBlock;

Review Comment:
   This should also be remaining == 0 right?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r937399932


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -53,10 +63,14 @@ public AesGcmEncryptor(byte[] keyBytes) {
       this.randomGenerator = new SecureRandom();
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {

Review Comment:
   @rdblue I've created a new PR, with the spec doc: https://github.com/apache/iceberg/pull/5432



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1039891570


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+
+public class AesGcmInputFile implements InputFile {

Review Comment:
   You're right. This would be the result of the `decrypt` method.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1039944476


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);

Review Comment:
   I thought this would be the block size, but it looks like the encrypted block size includes the nonce and tag. I think that's a bit confusing and we should make sure the behavior is specified in the spec.
   
   Since we're planning on making the block size constant, that works well. We can have a constant for this so it's easy.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1041945463


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remaining = len;

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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#issuecomment-1706217206

   Cool! LGTM. This could have been detected via external check of details (such as the file length, stored in parent metadata), but I agree doing this in the stream is cleaner and more reliable. Thanks for the PR, merged.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292845523


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {

Review Comment:
   This is really just to sanity check the state and avoid errors. The code is correct today because your assumption holds, but as this class changes, we want to make as many assumptions as possible explicit rather than relying on people knowing what to do.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292845058


##########
core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java:
##########
@@ -51,6 +53,21 @@ private void testEncryptDecrypt(byte[] aad) {
       Ciphers.AesGcmDecryptor decryptor = new Ciphers.AesGcmDecryptor(key);
       byte[] decryptedText = decryptor.decrypt(ciphertext, aad);
       assertThat(decryptedText).as("Key length " + keyLength).isEqualTo(plaintext);
+
+      // Test bad aad

Review Comment:
   This needs to be a separate test case / method.
   
   When you add additional cases to an existing test method, the tests are harder to work with because you don't get accurate descriptions of what failed. Plus, if there are issues with the case above, this test code won't be run so the person trying to figure out why tests are failing gets less information. Multiple failing cases are helpful for determining what needs to be fixed and for finding a good fix the first time that doesn't cause other failures.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278601655


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -101,37 +117,52 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad) {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(
+        byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) {
       Preconditions.checkState(
-          plainTextLength >= 1,
+          ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length "
               + ciphertext.length
               + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted"
               + " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);
 
-      byte[] plainText = new byte[plainTextLength];
-      int inputLength = ciphertext.length - NONCE_LENGTH;
+      int inputLength = ciphertextLength - NONCE_LENGTH;

Review Comment:
   This looks okay, but I'd want to double check how the tag bytes are handled. Here, the length includes the tag bytes so this isn't the same as the plaintext length that is used for encryption.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278604048


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+    currentDecryptedBlock = null;
+  }
+
+  static long calculatePlaintextLength(long sourceLength, int plainBlockSize) {
+    long netSourceFileLength = netSourceFileLength(sourceLength);
+
+    if (netSourceFileLength == 0) {
+      return 0;
+    }
+
+    int cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+    int numberOfFullBlocks = Math.toIntExact(netSourceFileLength / cipherBlockSize);
+    int cipherBytesInLastBlock =
+        Math.toIntExact(netSourceFileLength - numberOfFullBlocks * cipherBlockSize);

Review Comment:
   Since the stream length is a long, I don't think it makes sense to make `numberOfFullBlocks` in int. The multiplication can clearly produce a long anyway so this should remove the `toIntExact` above. Then everything is a long until `cipherBytesInLastBlock`.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904708479


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);

Review Comment:
   interesting though, the `available` doesn't have to be accurate, according to the InputStream contract. 
   `Returns an estimate of the number of bytes that can be read (or skipped over) from this input stream without blocking... while some implementations of InputStream will return the total number of bytes in the stream, many will not. It is never correct to use the return value of this method to allocate a buffer intended to hold all data in this stream.`
   
   I presume this means that we don't need to use Math.toIntExact since it throws an exception if the result can't be mapped to an int. I'll change the logic to return max int in that case.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904583638


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }
+
+      // Copy nonce
+      System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH);
+
+      byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock;
+      try {
+        GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce);
+        gcmCipher.init(Cipher.DECRYPT_MODE, key, spec);
+        gcmCipher.updateAAD(aad);
+
+        plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH,
+                toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH);
+      } catch (GeneralSecurityException e) {
+        throw new IOException("Failed to decrypt", e);
+      }
+
+      int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remaining >= remainingInBlock;

Review Comment:
   remaining is also unencrypted. If it's longer than what's left in current block, we'll read the block to the end



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044706354


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");

Review Comment:
   If we know from the file size that the content is 0 bytes, we don't need to use an encrypted stream. I think we should fail if the header isn't there and something tries to read the file, but if there are no encrypted blocks then we can just return a simpler stream.
   
   That will get rid of the `emptyCipherStream` cases and simplify this implementation.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044723489


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;

Review Comment:
   I think most Iceberg streams throw `EOFException` instead of returning -1.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044720784


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {

Review Comment:
   I think that this can be removed when the block size is set to a constant.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r931825742


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -19,19 +19,29 @@
 
 package org.apache.iceberg.encryption;
 
+import java.nio.charset.StandardCharsets;
 import java.security.GeneralSecurityException;
 import java.security.SecureRandom;
 import javax.crypto.AEADBadTagException;
 import javax.crypto.Cipher;
 import javax.crypto.spec.GCMParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.primitives.Ints;
 
 public class Ciphers {
-  private static final int NONCE_LENGTH = 12;
-  private static final int GCM_TAG_LENGTH = 16;
+  public static final int NONCE_LENGTH = 12;
+  public static final int GCM_TAG_LENGTH = 16;
+  public static final String GCM_STREAM_MAGIC_STRING = "GCM1";

Review Comment:
   The GCM Streams are introduced in this PR - being proposed as a part of the Iceberg spec / standard. 



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1041943937


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];

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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278601932


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -101,37 +117,52 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad) {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(

Review Comment:
   I think that `TestCiphers` needs to have tests validating failure modes:
   * If the supplied AAD is incorrect, decryption fails
   * If the ciphertext is modified, decryption fails
   
   Right now, only the happy path is tested, which doesn't validate the intended authentication behavior using the tag.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278601003


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -67,7 +83,7 @@ public byte[] encrypt(byte[] plainText, byte[] aad) {
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
+        cipher.doFinal(plaintext, plaintextOffset, plaintextLength, cipherText, NONCE_LENGTH);

Review Comment:
   Should this check that the number of bytes returned by `doFinal` is correct?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278951980


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -101,37 +117,52 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad) {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(
+        byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) {
       Preconditions.checkState(
-          plainTextLength >= 1,
+          ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length "
               + ciphertext.length
               + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted"
               + " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);

Review Comment:
   Cool!



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1281539601


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -55,10 +71,15 @@ public AesGcmEncryptor(byte[] keyBytes) {
       this.randomGenerator = new SecureRandom();
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
       byte[] nonce = new byte[NONCE_LENGTH];
       randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;

Review Comment:
   Added to both encryption and decryption in the new commit.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1264953929


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -96,33 +110,45 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
-      Preconditions.checkState(plainTextLength >= 1,
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad)  {

Review Comment:
   Unfortunately, this change triggers
   
   > Execution failed for task ':iceberg-core:revapi'.
   > > There were Java public API/ABI breaks reported by revapi:
   >   java.method.removed: Method was removed.
   
   How this should be handled?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1265042174


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");

Review Comment:
   I've tried this, and ran into the following issue. If the execution has reached this place, it means we expect an encrypted file (for example - reading manifests with non-null key_metadata). Maybe we should be stricter here, and simply throw an exception for empty cipher streams; these classes are designed for encryption of non-empty content; also, this will help to detect wrong situations earlier. What do you think?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#issuecomment-1705665555

   @ggershinsky, I opened https://github.com/ggershinsky/iceberg/pull/8 with the remaining changes for this PR.
   
   The main change is to write an empty block for empty files. Otherwise an attacker could replace a valid file with an empty file without detection.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1275712542


##########
.palantir/revapi.yml:
##########
@@ -743,6 +743,12 @@ acceptedBreaks:
       new: "method java.util.List<org.apache.iceberg.UpdateRequirement> org.apache.iceberg.rest.requests.UpdateTableRequest::requirements()"
       justification: "Signature changed to an interface, but this is safe because\
         \ of type erasure and the original type is always returned"
+  "1.3.0":
+    org.apache.iceberg:iceberg-core:
+    - code: "java.method.visibilityReduced"
+      old: "method void org.apache.iceberg.encryption.Ciphers::<init>()"
+      new: "method void org.apache.iceberg.encryption.Ciphers::<init>()"
+      justification: "Static utility class - should not have public constructor"

Review Comment:
   Yep, the CI is failing unfortunately, since util classes are now required to have private constructors.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292843203


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,138 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] fileAadPrefix;
+  private final byte[] singleByte;
+
+  private byte[] plainBlock;
+  private byte[] cipherBlock;
+  private int positionInPlainBlock;
+  private long streamPosition;
+  private int currentBlockIndex;
+  private boolean isHeaderWritten;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlock = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.cipherBlock = new byte[Ciphers.CIPHER_BLOCK_SIZE];
+    this.positionInPlainBlock = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+    this.isHeaderWritten = false;
+    this.singleByte = new byte[1];
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    singleByte[0] = (byte) (b & 0x000000FF);
+    write(singleByte);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (!isHeaderWritten) {
+      writeHeader();
+    }
+
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = plainBlock.length - positionInPlainBlock;
+      int toWrite = Math.min(freeBlockBytes, remaining);
+
+      System.arraycopy(b, offset, plainBlock, positionInPlainBlock, toWrite);
+      positionInPlainBlock += toWrite;
+      offset += toWrite;
+      remaining -= toWrite;
+
+      if (positionInPlainBlock == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+      }
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (!isHeaderWritten) {
+      writeHeader();
+    }
+
+    if (positionInPlainBlock > 0) {
+      encryptAndWriteBlock();
+    }
+
+    targetStream.close();
+    plainBlock = null;
+    cipherBlock = null;
+  }
+
+  private void writeHeader() throws IOException {
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();

Review Comment:
   This byte array is made entirely from constants. Can you make it a constant byte array?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292843533


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -55,10 +71,15 @@ public AesGcmEncryptor(byte[] keyBytes) {
       this.randomGenerator = new SecureRandom();
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
       byte[] nonce = new byte[NONCE_LENGTH];
       randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;

Review Comment:
   I think we should avoid allocation on every `encrypt` call. Can you make it so that the stream passes this in and reuses it?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904351383


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }
+
+      // Copy nonce
+      System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH);
+
+      byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock;
+      try {
+        GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce);
+        gcmCipher.init(Cipher.DECRYPT_MODE, key, spec);
+        gcmCipher.updateAAD(aad);
+
+        plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH,
+                toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH);
+      } catch (GeneralSecurityException e) {
+        throw new IOException("Failed to decrypt", e);
+      }
+
+      int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remaining >= remainingInBlock;
+      int toCopy = finishTheBlock ? remainingInBlock : remaining;
+
+      System.arraycopy(plaintextBlock, currentOffsetInPlainBlock, b, resultBufferOffset, toCopy);
+      remaining -= toCopy;
+      resultBufferOffset += toCopy;
+      currentOffsetInPlainBlock += toCopy;
+      boolean endOfStream = lastBlock && finishTheBlock;
+      if (endOfStream) {
+        break;
+      }
+      if (finishTheBlock) {

Review Comment:
   Could we change skip the break and just have it be 
   ```java
   if (finishTheBlock && !endOfStream)
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904098268


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {

Review Comment:
   Precondition?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902973604


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.SecureRandom;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.primitives.Ints;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+  // AES-GCM parameters
+  public static final int GCM_NONCE_LENGTH = 12; // in bytes
+  public static final int GCM_TAG_LENGTH = 16; // in bytes
+  public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+  public static final String MAGIC_STRING = "GCM1";
+
+  static final byte[] MAGIC_ARRAY = MAGIC_STRING.getBytes(StandardCharsets.UTF_8);
+  static final int PREFIX_LENGTH = MAGIC_ARRAY.length + 4; // magic_len + block_size_len
+
+  private PositionOutputStream targetStream;
+
+  private Cipher gcmCipher;
+  private SecureRandom random;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private int blockSize = 1024 * 1024;
+  private byte[] plaintextBlockBuffer;
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+  private byte[] fileAadPrefix;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.targetStream = targetStream;
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.random = new SecureRandom();
+    this.nonce = new byte[GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.plaintextBlockBuffer = new byte[blockSize];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] prefixBytes = ByteBuffer.allocate(PREFIX_LENGTH).order(ByteOrder.LITTLE_ENDIAN)
+        .put(MAGIC_ARRAY)
+        .putInt(blockSize)
+        .array();
+    targetStream.write(prefixBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b)  throws IOException {
+    write(b, 0, b.length);
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = blockSize - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plaintextBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == blockSize) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {
+    random.nextBytes(nonce);
+    GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
+    try {
+      gcmCipher.init(Cipher.ENCRYPT_MODE, key, spec);
+    } catch (InvalidKeyException | InvalidAlgorithmParameterException e) {
+      throw new IOException("Failed to init GCM cipher", e);
+    }
+
+    byte[] aad = calculateAAD(fileAadPrefix, currentBlockIndex);
+    gcmCipher.updateAAD(aad);
+
+    byte[] cipherText = new byte[GCM_NONCE_LENGTH + positionInBuffer + GCM_TAG_LENGTH];
+    System.arraycopy(nonce, 0, cipherText, 0, GCM_NONCE_LENGTH);
+    try {
+      int encrypted = gcmCipher.doFinal(plaintextBlockBuffer, 0, positionInBuffer, cipherText, GCM_NONCE_LENGTH);
+      Preconditions.checkArgument((encrypted == (positionInBuffer + GCM_TAG_LENGTH)),
+          "Wrong length of encrypted output: " + encrypted + " vs " + (positionInBuffer + GCM_TAG_LENGTH));

Review Comment:
   How about a message like this?
   ```
   "Wrong length of encrypted output:  expected " + (positionInBuffer + GCM_TAG_LENGTH)) + " but " + encrypted;
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902971900


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.SecureRandom;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.primitives.Ints;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+  // AES-GCM parameters
+  public static final int GCM_NONCE_LENGTH = 12; // in bytes
+  public static final int GCM_TAG_LENGTH = 16; // in bytes
+  public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+  public static final String MAGIC_STRING = "GCM1";
+
+  static final byte[] MAGIC_ARRAY = MAGIC_STRING.getBytes(StandardCharsets.UTF_8);
+  static final int PREFIX_LENGTH = MAGIC_ARRAY.length + 4; // magic_len + block_size_len
+
+  private PositionOutputStream targetStream;
+
+  private Cipher gcmCipher;
+  private SecureRandom random;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private int blockSize = 1024 * 1024;

Review Comment:
   Make it a final static field?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292843533


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -55,10 +71,15 @@ public AesGcmEncryptor(byte[] keyBytes) {
       this.randomGenerator = new SecureRandom();
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
       byte[] nonce = new byte[NONCE_LENGTH];
       randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;

Review Comment:
   I think we should avoid allocation on every `encrypt` call. Can you make it so that the stream passes this in and reuses it?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1293160114


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {

Review Comment:
   In my previous comment, I forgot about another call of this function - from the stream `close()` method. There, the last block is not assumed to be full, since this writes the remains of the plainBlock buffer.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue merged pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue merged PR #3231:
URL: https://github.com/apache/iceberg/pull/3231


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278604247


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+    currentDecryptedBlock = null;
+  }
+
+  static long calculatePlaintextLength(long sourceLength, int plainBlockSize) {
+    long netSourceFileLength = netSourceFileLength(sourceLength);
+
+    if (netSourceFileLength == 0) {
+      return 0;
+    }
+
+    int cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+    int numberOfFullBlocks = Math.toIntExact(netSourceFileLength / cipherBlockSize);
+    int cipherBytesInLastBlock =
+        Math.toIntExact(netSourceFileLength - numberOfFullBlocks * cipherBlockSize);
+    boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+    int plainBytesInLastBlock =
+        fullBlocksOnly
+            ? 0
+            : (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+
+    return (long) numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;

Review Comment:
   Using a `long` for `numberOfFullBlocks` avoids the need for a cast here. Also, since this is producing a `long`, I don't think it makes sense to do the extra logic to keep either `cipherBytesInLastBlock` or `plainBytesInLastBlock` as integers. Just use longs and not worry about the casts.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278598615


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,77 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+
+public class AesGcmInputFile implements InputFile {
+  private final InputFile sourceFile;
+  private final byte[] dataKey;
+  private final byte[] fileAADPrefix;
+  private long plaintextLength;
+
+  public AesGcmInputFile(InputFile sourceFile, byte[] dataKey, byte[] fileAADPrefix) {
+    this.sourceFile = sourceFile;
+    this.dataKey = dataKey;
+    this.fileAADPrefix = fileAADPrefix;
+    this.plaintextLength = -1;
+  }
+
+  @Override
+  public long getLength() {
+    if (plaintextLength == -1) {
+      // Presumes all streams use hard-coded plaintext block size.
+      // Actual plaintext block size is checked upon stream creation (exception if different).
+      plaintextLength =
+          AesGcmInputStream.calculatePlaintextLength(
+              sourceFile.getLength(), AesGcmOutputStream.plainBlockSize);
+    }
+
+    return plaintextLength;
+  }
+
+  @Override
+  public SeekableInputStream newStream() {
+    getLength();

Review Comment:
   Why is this called? It isn't used anywhere.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1269060144


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new IOException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);

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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1054069364


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,

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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1054066094


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -96,33 +110,45 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
-      Preconditions.checkState(plainTextLength >= 1,
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad)  {
+      Preconditions.checkState(ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length " + ciphertext.length +
           " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" +
           " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);
 
-      byte[] plainText = new byte[plainTextLength];
-      int inputLength = ciphertext.length - NONCE_LENGTH;
+      int inputLength = ciphertextLength - NONCE_LENGTH;
       try {
         GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
         cipher.init(Cipher.DECRYPT_MODE, aesKey, spec);
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(ciphertext, NONCE_LENGTH, inputLength, plainText, 0);
+        return cipher.doFinal(ciphertext, ciphertextOffset + NONCE_LENGTH, inputLength);
       }  catch (AEADBadTagException e) {
         throw new RuntimeException("GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" +
-            "data. AES GCM doesn't differentiate between these two.. ", e);
+            "data. AES GCM doesn't differentiate between these two.", e);
       } catch (GeneralSecurityException e) {
         throw new RuntimeException("Failed to decrypt", e);
       }
+    }
+  }
 
-      return plainText;
+  static byte[] streamBlockAAD(byte[] fileAadPrefix, int currentBlockIndex) {
+    byte[] blockAAD = Ints.toByteArray(currentBlockIndex);

Review Comment:
   Agreed, we'll use the same endianness.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1054070560


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;

Review Comment:
   I see, lets do the same here.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044708947


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,

Review Comment:
   This conclusion isn't correct. While unlikely to happen, the underlying stream is allowed to return fewer than the requested bytes. That isn't an error case and it doesn't indicate that the stream hit EOF.
   
   If you want to ensure that you get all the bytes requested, then use `IOUtil.readFully`. That will throw `EOFException` if there aren't enough bytes.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044743460


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(Ciphers.GCM_STREAM_PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastCipherBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Should read " + toLoad + " bytes, but got only " + loaded + " bytes");
+      }
+
+      byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock = gcmDecryptor.decrypt(ciphertextBlockBuffer, 0, toLoad, aad);

Review Comment:
   It looks to me like this is going to read and decrypt the current block every time `read` is called. Am I wrong? Why not keep the current decrypted content in a buffer that persists across calls and consume bytes as needed from that buffer?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044792391


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -96,33 +110,45 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
-      Preconditions.checkState(plainTextLength >= 1,
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad)  {
+      Preconditions.checkState(ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length " + ciphertext.length +
           " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" +
           " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);
 
-      byte[] plainText = new byte[plainTextLength];
-      int inputLength = ciphertext.length - NONCE_LENGTH;
+      int inputLength = ciphertextLength - NONCE_LENGTH;
       try {
         GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
         cipher.init(Cipher.DECRYPT_MODE, aesKey, spec);
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(ciphertext, NONCE_LENGTH, inputLength, plainText, 0);
+        return cipher.doFinal(ciphertext, ciphertextOffset + NONCE_LENGTH, inputLength);
       }  catch (AEADBadTagException e) {
         throw new RuntimeException("GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" +
-            "data. AES GCM doesn't differentiate between these two.. ", e);
+            "data. AES GCM doesn't differentiate between these two.", e);
       } catch (GeneralSecurityException e) {
         throw new RuntimeException("Failed to decrypt", e);
       }
+    }
+  }
 
-      return plainText;
+  static byte[] streamBlockAAD(byte[] fileAadPrefix, int currentBlockIndex) {
+    byte[] blockAAD = Ints.toByteArray(currentBlockIndex);

Review Comment:
   Why use big endian when other places use little endian?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904598946


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }
+
+      // Copy nonce
+      System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH);
+
+      byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock;
+      try {
+        GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce);
+        gcmCipher.init(Cipher.DECRYPT_MODE, key, spec);
+        gcmCipher.updateAAD(aad);
+
+        plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH,
+                toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH);
+      } catch (GeneralSecurityException e) {
+        throw new IOException("Failed to decrypt", e);
+      }
+
+      int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remaining >= remainingInBlock;
+      int toCopy = finishTheBlock ? remainingInBlock : remaining;
+
+      System.arraycopy(plaintextBlock, currentOffsetInPlainBlock, b, resultBufferOffset, toCopy);
+      remaining -= toCopy;
+      resultBufferOffset += toCopy;
+      currentOffsetInPlainBlock += toCopy;
+      boolean endOfStream = lastBlock && finishTheBlock;
+      if (endOfStream) {
+        break;
+      }
+      if (finishTheBlock) {

Review Comment:
   we need the break to stop the loop, no need to continue reading if the stream has ended.
   finishTheBlock simply means we've finished one block, and start reading the next.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044726532


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(Ciphers.GCM_STREAM_PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastCipherBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Should read " + toLoad + " bytes, but got only " + loaded + " bytes");

Review Comment:
   This isn't an exceptional case. If you need to load the entire buffer, use `IOUtils.readFully`.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044724923


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);

Review Comment:
   I think this should be an `IllegalArgumentException`. Can you use a precondition?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1041939078


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+
+public class AesGcmInputFile implements InputFile {
+  private final InputFile sourceFile;
+  private final byte[] dataKey;
+  private long plaintextLength;
+
+  public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) {
+    this.sourceFile = sourceFile;
+    this.dataKey = dataKey;
+    this.plaintextLength = -1;
+  }
+
+  @Override
+  public long getLength() {
+    if (plaintextLength == -1) {
+      try {
+        this.newStream().close();

Review Comment:
   Per our recent discussion, we'll set the default block size to 1MB, so the plaintext length can be calculated without opening the stream.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1054071435


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);

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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904026316


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);

Review Comment:
   Aren't we also in danger of the read being less than prefix bytes?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904022823


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+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 InputFile sourceFile;
+  private byte[] dataKey;
+  private long plaintextLength;
+
+  public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) {
+    this.sourceFile = sourceFile;
+    this.dataKey = dataKey;
+    this.plaintextLength = -1;
+  }
+
+  @Override
+  public long getLength() {
+    Preconditions.checkArgument(plaintextLength >= 0, "Length is known after new stream is created");

Review Comment:
   When do we close the stream in this case? Or do we just make and close it in that case?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904075069


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");

Review Comment:
   Constant for this?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r929262253


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -96,33 +110,45 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
-      Preconditions.checkState(plainTextLength >= 1,
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad)  {
+      Preconditions.checkState(ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length " + ciphertext.length +
           " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" +
           " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);
 
-      byte[] plainText = new byte[plainTextLength];
-      int inputLength = ciphertext.length - NONCE_LENGTH;
+      int inputLength = ciphertextLength - NONCE_LENGTH;
       try {
         GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
         cipher.init(Cipher.DECRYPT_MODE, aesKey, spec);
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(ciphertext, NONCE_LENGTH, inputLength, plainText, 0);
+        return cipher.doFinal(ciphertext, ciphertextOffset + NONCE_LENGTH, inputLength);
       }  catch (AEADBadTagException e) {
         throw new RuntimeException("GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" +
-            "data. AES GCM doesn't differentiate between these two.. ", e);
+            "data. AES GCM doesn't differentiate between these two.", e);
       } catch (GeneralSecurityException e) {
         throw new RuntimeException("Failed to decrypt", e);
       }
+    }
+  }
 
-      return plainText;
+  static byte[] streamBlockAAD(byte[] fileAadPrefix, int currentBlockIndex) {
+    byte[] blockAAD = Ints.toByteArray(currentBlockIndex);

Review Comment:
   I don't think this is specific enough. What is happening here? What is the endianness?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902972234


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,173 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.nio.charset.StandardCharsets;
+import java.security.GeneralSecurityException;
+import java.security.InvalidAlgorithmParameterException;
+import java.security.InvalidKeyException;
+import java.security.SecureRandom;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.primitives.Ints;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+  // AES-GCM parameters
+  public static final int GCM_NONCE_LENGTH = 12; // in bytes
+  public static final int GCM_TAG_LENGTH = 16; // in bytes
+  public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+  public static final String MAGIC_STRING = "GCM1";
+
+  static final byte[] MAGIC_ARRAY = MAGIC_STRING.getBytes(StandardCharsets.UTF_8);
+  static final int PREFIX_LENGTH = MAGIC_ARRAY.length + 4; // magic_len + block_size_len
+
+  private PositionOutputStream targetStream;
+
+  private Cipher gcmCipher;
+  private SecureRandom random;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private int blockSize = 1024 * 1024;
+  private byte[] plaintextBlockBuffer;
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+  private byte[] fileAadPrefix;

Review Comment:
   It can be a final field.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1279976921


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputFile implements OutputFile {
+  private final OutputFile targetFile;
+  private final byte[] dataKey;
+  private final byte[] fileAADPrefix;
+
+  public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey, byte[] fileAADPrefix) {
+    this.targetFile = targetFile;
+    this.dataKey = dataKey;
+    this.fileAADPrefix = fileAADPrefix;
+  }
+
+  @Override
+  public PositionOutputStream create() {
+    try {
+      return new AesGcmOutputStream(targetFile.create(), dataKey, fileAADPrefix);
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to create GCM stream for " + targetFile.location(), e);
+    }
+  }
+
+  @Override
+  public PositionOutputStream createOrOverwrite() {
+    try {
+      return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, fileAADPrefix);
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          "Failed to create or overwrite GCM stream for " + targetFile.location(), e);
+    }
+  }
+
+  @Override
+  public String location() {
+    return targetFile.location();
+  }
+
+  @Override
+  public InputFile toInputFile() {
+    throw new UnsupportedOperationException();

Review Comment:
   This should be implemented.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1279976178


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputFile implements OutputFile {
+  private final OutputFile targetFile;
+  private final byte[] dataKey;
+  private final byte[] fileAADPrefix;
+
+  public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey, byte[] fileAADPrefix) {
+    this.targetFile = targetFile;
+    this.dataKey = dataKey;
+    this.fileAADPrefix = fileAADPrefix;
+  }
+
+  @Override
+  public PositionOutputStream create() {
+    try {
+      return new AesGcmOutputStream(targetFile.create(), dataKey, fileAADPrefix);
+    } catch (IOException e) {

Review Comment:
   Similar to my [note above](https://github.com/apache/iceberg/pull/3231/files#r1278605737), I don't think that constructors should throw IOException.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1279975190


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,264 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final byte[] fileAADPrefix;
+  private final Ciphers.AesGcmDecryptor decryptor;
+  private final byte[] cipherBlockBuffer;
+  private final long numBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+
+  private long plainStreamPosition;
+  private long currentPlainBlockIndex;
+  private byte[] currentPlainBlock;
+  private int currentPlainBlockSize;
+  private byte[] singleByte;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix) {
+    this.sourceStream = sourceStream;
+    this.fileAADPrefix = fileAADPrefix;
+    this.decryptor = new Ciphers.AesGcmDecryptor(aesKey);
+    this.cipherBlockBuffer = new byte[Ciphers.CIPHER_BLOCK_SIZE];
+
+    this.plainStreamPosition = 0;
+    this.currentPlainBlockIndex = -1;
+    this.currentPlainBlock = null;
+    this.currentPlainBlockSize = 0;
+
+    long streamLength = sourceLength - Ciphers.GCM_STREAM_HEADER_LENGTH;
+    long numFullBlocks = Math.toIntExact(streamLength / Ciphers.CIPHER_BLOCK_SIZE);
+    long cipherFullBlockLength = numFullBlocks * Ciphers.CIPHER_BLOCK_SIZE;
+    int cipherBytesInLastBlock = Math.toIntExact(streamLength - cipherFullBlockLength);
+    boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+    this.numBlocks = fullBlocksOnly ? numFullBlocks : numFullBlocks + 1;
+    this.lastCipherBlockSize =
+        fullBlocksOnly ? Ciphers.CIPHER_BLOCK_SIZE : cipherBytesInLastBlock; // never 0
+
+    long lastPlainBlockSize =
+        (long) lastCipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH;
+    this.plainStreamSize =
+        numFullBlocks * Ciphers.PLAIN_BLOCK_SIZE + (fullBlocksOnly ? 0 : lastPlainBlockSize);
+    this.singleByte = new byte[1];
+  }
+
+  private void validateHeader() throws IOException {
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+
+    Preconditions.checkState(
+        Ciphers.GCM_STREAM_MAGIC.equals(ByteBuffer.wrap(headerBytes, 0, 4)),
+        "Invalid GCM stream: magic does not match AGS1");
+
+    int plainBlockSize = ByteBuffer.wrap(headerBytes, 4, 4).order(ByteOrder.LITTLE_ENDIAN).getInt();
+    Preconditions.checkState(
+        plainBlockSize == Ciphers.PLAIN_BLOCK_SIZE,
+        "Invalid GCM stream: block size %d != %d",
+        plainBlockSize,
+        Ciphers.PLAIN_BLOCK_SIZE);
+  }
+
+  @Override
+  public int available() {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  private int availableInCurrentBlock() {
+    if (currentPlainBlockIndex < 0) {

Review Comment:
   When I was refactoring, there was a bug that I fixed when the block for `plainStreamPosition` didn't match `currentPlainBlockIndex`. I ended up fixing the problem by setting `currentPlainBlockIndex = -1` in a place that I had missed. Now that I'm thinking about it more, I think the right solution is actually to update the check here instead. That way the bytes available will only be non-zero if the current block matches the position.
   
   Here's a diff that does what I'm talking about and still passes tests:
   
   ```diff
   diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java
   index a63134f31d..88e9b36c25 100644
   --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java
   +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java
   @@ -97,7 +97,7 @@ public class AesGcmInputStream extends SeekableInputStream {
      }
    
      private int availableInCurrentBlock() {
   -    if (currentPlainBlockIndex < 0) {
   +    if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) {
          return 0;
        }
    
   @@ -130,10 +130,6 @@ public class AesGcmInputStream extends SeekableInputStream {
            remainingBytesToRead -= bytesToCopy;
            resultBufferOffset += bytesToCopy;
            this.plainStreamPosition += bytesToCopy;
   -        if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) {
   -          // invalidate the current block
   -          this.currentPlainBlockIndex = -1;
   -        }
    
          } else if (available() > 0) {
            decryptBlock(blockIndex(plainStreamPosition));
   @@ -157,10 +153,6 @@ public class AesGcmInputStream extends SeekableInputStream {
        }
    
        this.plainStreamPosition = newPos;
   -    if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) {
   -      // invalidate the current block
   -      this.currentPlainBlockIndex = -1;
   -    }
      }
    
      @Override
   @@ -177,10 +169,6 @@ public class AesGcmInputStream extends SeekableInputStream {
        }
    
        this.plainStreamPosition += n;
   -    if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) {
   -      // invalidate the current block
   -      this.currentPlainBlockIndex = -1;
   -    }
    
        return n;
      }
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278605345


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];

Review Comment:
   This allocation isn't needed. You can wrap the first 4 bytes of the header array in a ByteBuffer and use that for the comparison. That is also better to that you don't need to expose a mutable array in Ciphers:
   
   ```java
     static final ByteBuffer GCM_STREAM_MAGIC = ByteBuffer.wrap(GCM_STREAM_MAGIC_ARRAY).asReadOnlyBuffer();
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278604430


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+    currentDecryptedBlock = null;
+  }
+
+  static long calculatePlaintextLength(long sourceLength, int plainBlockSize) {
+    long netSourceFileLength = netSourceFileLength(sourceLength);
+
+    if (netSourceFileLength == 0) {
+      return 0;
+    }
+
+    int cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+    int numberOfFullBlocks = Math.toIntExact(netSourceFileLength / cipherBlockSize);
+    int cipherBytesInLastBlock =
+        Math.toIntExact(netSourceFileLength - numberOfFullBlocks * cipherBlockSize);
+    boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+    int plainBytesInLastBlock =
+        fullBlocksOnly
+            ? 0
+            : (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+
+    return (long) numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;

Review Comment:
   Here's the full method with my changes to types:
   
   ```java
     static long calculatePlaintextLength(long sourceLength) {
       long streamLength = sourceLength - Ciphers.GCM_STREAM_HEADER_LENGTH;
   
       if (streamLength == 0) {
         return 0;
       }
   
       long numberOfFullBlocks = streamLength / AesGcmOutputStream.CIPHER_BLOCK_SIZE;
       long fullBlockSize = numberOfFullBlocks * AesGcmOutputStream.CIPHER_BLOCK_SIZE;
       long cipherBytesInLastBlock = streamLength - fullBlockSize;
       boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
       long plainBytesInLastBlock =
           fullBlocksOnly
               ? 0
               : (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
   
       return (numberOfFullBlocks * AesGcmOutputStream.PLAIN_BLOCK_SIZE) + plainBytesInLastBlock;
     }
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278603093


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+    currentDecryptedBlock = null;
+  }
+
+  static long calculatePlaintextLength(long sourceLength, int plainBlockSize) {
+    long netSourceFileLength = netSourceFileLength(sourceLength);
+
+    if (netSourceFileLength == 0) {
+      return 0;

Review Comment:
   @ggershinsky, I think this introduces a deletion attack.
   
   I think this means that you can have a 0-length content file. That might make sense, but if we are going to allow a 0-length file then it should actually use a 0-length block as well.
   
   The problem with a 0-length stream is that an attacker can replace a file with a 0-length stream to effectively delete data while bypassing authentication checks. A 0-length block would prevent that attack because it would check the AAD for the block.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278604464


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();

Review Comment:
   This is required.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278604487


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;

Review Comment:
   Style: missing newline after the previous control flow block.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278959414


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -101,37 +117,52 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad) {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(
+        byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) {
       Preconditions.checkState(
-          plainTextLength >= 1,
+          ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length "
               + ciphertext.length
               + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted"
               + " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);
 
-      byte[] plainText = new byte[plainTextLength];
-      int inputLength = ciphertext.length - NONCE_LENGTH;
+      int inputLength = ciphertextLength - NONCE_LENGTH;

Review Comment:
   Correct. 
   I removed this var, its name is not intuitive enough.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1281430467


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;

Review Comment:
   this value is returned by 
   
   ```java
   @Override
     public long getPos() throws IOException {
       return streamPosition;
     }
   ```
   
     so is not related directly to targetStream.pos() and positionInBuffer.
   I'll rename `positionInBuffer` to `positionInPlainBlock`, and `plainBlockBuffer` to `plainBlock`, to make the naming more consistent.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1268651697


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new IOException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);

Review Comment:
   Shouldn't this be EOFException?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278917331


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -67,7 +83,7 @@ public byte[] encrypt(byte[] plainText, byte[] aad) {
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
+        cipher.doFinal(plaintext, plaintextOffset, plaintextLength, cipherText, NONCE_LENGTH);

Review Comment:
   > Does the call to getInstance in the constructor guarantee that the Cipher instance is not shared? 
   
   The `AesGcmEncryptor` and `AesGcmDecryptor` are designed for a single thread use.
   
   > Should this check that the number of bytes returned by doFinal is correct?
   
   Sure, we can check this.
   
   > Can you confirm that doFinal also writes the GCM tag to the output buffer?
   
   From the `Cipher.doFinal` javadoc: 
    _If an AEAD mode such as GCM/CCM is being used, the authentication tag is appended in the case of encryption, or verified in the case of decryption._



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1281462322


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {

Review Comment:
   this function is called only for full blocks, after an explicit check:
   
   ```java
   if (positionInPlainBlock == Ciphers.PLAIN_BLOCK_SIZE) {
           encryptAndWriteBlock();
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278602668


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+    currentDecryptedBlock = null;
+  }
+
+  static long calculatePlaintextLength(long sourceLength, int plainBlockSize) {
+    long netSourceFileLength = netSourceFileLength(sourceLength);
+
+    if (netSourceFileLength == 0) {
+      return 0;
+    }
+
+    int cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+    int numberOfFullBlocks = Math.toIntExact(netSourceFileLength / cipherBlockSize);
+    int cipherBytesInLastBlock =
+        Math.toIntExact(netSourceFileLength - numberOfFullBlocks * cipherBlockSize);
+    boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+    int plainBytesInLastBlock =
+        fullBlocksOnly
+            ? 0
+            : (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+
+    return (long) numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+  }
+
+  private byte[] decryptNextBlock(boolean isLastBlockInStream) throws IOException {
+    if (currentBlockIndex == currentDecryptedBlockIndex) {
+      return currentDecryptedBlock;
+    }
+
+    long blockPositionInStream = blockOffset(currentBlockIndex);
+    if (sourceStream.getPos() != blockPositionInStream) {
+      sourceStream.seek(blockPositionInStream);
+    }
+
+    int currentCipherBlockSize = isLastBlockInStream ? lastCipherBlockSize : cipherBlockSize;
+    IOUtil.readFully(sourceStream, cipherBlockBuffer, 0, currentCipherBlockSize);
+
+    byte[] aad = Ciphers.streamBlockAAD(fileAADPrefix, currentBlockIndex);
+    byte[] result = gcmDecryptor.decrypt(cipherBlockBuffer, 0, currentCipherBlockSize, aad);
+    currentDecryptedBlockIndex = currentBlockIndex;
+    currentDecryptedBlock = result;
+    return result;
+  }
+
+  private long blockOffset(int blockIndex) {
+    return (long) blockIndex * cipherBlockSize + Ciphers.GCM_STREAM_HEADER_LENGTH;
+  }
+
+  private static long netSourceFileLength(long sourceFileLength) {

Review Comment:
   I don't think that "net" is clear enough, and I also think that this method is doing too much. A calculation should not include a Precondition check like this because it is surprising. You must know the implementation details of the method to call it properly.
   
   I think that the length should be validated in `newStream` when attempting to read the file, and _not_ when calculating the file length for `InputStream.getLength`.
   
   I think checking at that point avoids the need to have this method entirely, which avoids the naming problem. The call can simply be `encryptedFileLength - Ciphers.GCM_STREAM_HEADER_LENGTH`.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278600384


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -67,7 +83,7 @@ public byte[] encrypt(byte[] plainText, byte[] aad) {
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
+        cipher.doFinal(plaintext, plaintextOffset, plaintextLength, cipherText, NONCE_LENGTH);

Review Comment:
   Does the call to `getInstance` in the constructor guarantee that the `Cipher` instance is not shared? I'm guessing that it must if it has state (mode, key, etc.) but I don't see it guaranteed anywhere.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278606283


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);

Review Comment:
   This duplicates the next check.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278605737


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);

Review Comment:
   This should avoid reading in the constructor. Otherwise, this needs to throw IOException and it is harder for the caller to ensure that resources are closed if anything goes wrong.
   
   For example, if this throws `IOException` then `newStream` isn't going to return anything. This stream is never constructed, but this is the owner of `sourceFile.newStream()` that is passed in as `sourceStream`. As a result, `sourceStream` is leaked.
   
   Instead, this should lazily initialize and use something like the `currentBlockIndex` to keep track of whether it needs to init.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#issuecomment-1657273433

   @ggershinsky, I had a lot of comments, so I went ahead and implemented most of them on top of this. I think it simplified how the input stream works quite a bit and I fixed a lot of the comments that I made. Please see https://github.com/ggershinsky/iceberg/pull/6.
   
   I think we'll also need to apply similar changes to the output stream, but I haven't had a chance to take a close look yet.


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904043987


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);

Review Comment:
   Instead of "file with wrong magic string" -> "Cannot read unencrypted file, missing header containing: ..."?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902970238


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }
+
+      // Copy nonce
+      System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH);
+
+      byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock;
+      try {
+        GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce);
+        gcmCipher.init(Cipher.DECRYPT_MODE, key, spec);
+        gcmCipher.updateAAD(aad);
+
+        plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH,
+                toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH);
+      } catch (GeneralSecurityException e) {
+        throw new IOException("Failed to decrypt", e);
+      }
+
+      int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remaining >= remainingInBlock;
+      int toCopy = finishTheBlock ? remainingInBlock : remaining;
+
+      System.arraycopy(plaintextBlock, currentOffsetInPlainBlock, b, resultBufferOffset, toCopy);
+      remaining -= toCopy;
+      resultBufferOffset += toCopy;
+      currentOffsetInPlainBlock += toCopy;
+      boolean endOfStream = lastBlock && finishTheBlock;
+      if (endOfStream) {
+        break;
+      }
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+      }
+    }
+
+    plainStreamPosition += len - remaining;
+    return len - remaining;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos >= plainStreamSize) {
+      throw new IOException("At or beyond max stream size " + plainStreamSize + ", " + newPos);
+    }
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+  }
+
+  @Override
+  public synchronized void mark(int readlimit) {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public synchronized void reset() throws IOException {
+    throw new UnsupportedOperationException();

Review Comment:
   Looks like we don't have to implement it, the abstract class `InputStream` has the same logic.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902968904


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);

Review Comment:
   Minor suggestion: Would it be a bit easier to understand if message is "Should read `toLoad` data, but only get `loaded` data"?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904337068


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;

Review Comment:
   This is part of the code I think is kind of complicated because of the special casing of the last block logic. Let's try to make this as clear as possible. 
   
   Do we actually need to keep these numbers here? I was thinking about this again, could we instead do something like
   ```java
   Math.min(cipherBlockSize, remaining)
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904529142


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,67 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+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 InputFile sourceFile;
+  private byte[] dataKey;
+  private long plaintextLength;
+
+  public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) {
+    this.sourceFile = sourceFile;
+    this.dataKey = dataKey;
+    this.plaintextLength = -1;
+  }
+
+  @Override
+  public long getLength() {
+    Preconditions.checkArgument(plaintextLength >= 0, "Length is known after new stream is created");

Review Comment:
   Both suggestions sound good to me.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1041949962


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");

Review Comment:
   This code can be reached only if the parent metadata has marked this file as encrypted (and provided us with the decryption key). If the file is not encrypted, this is an error situation.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r929261471


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -96,33 +110,45 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
-      Preconditions.checkState(plainTextLength >= 1,
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad)  {

Review Comment:
   Should this be `decryptBlock` instead?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1286776558


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -27,10 +30,19 @@
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 
 public class Ciphers {
-  private static final int NONCE_LENGTH = 12;
-  private static final int GCM_TAG_LENGTH = 16;
+  public static final int NONCE_LENGTH = 12;
+  public static final int GCM_TAG_LENGTH = 16;
+  public static final String GCM_STREAM_MAGIC_STRING = "AGS1";
+
+  static final byte[] GCM_STREAM_MAGIC_ARRAY =
+      GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8);
+  static final int GCM_STREAM_HEADER_LENGTH =
+      GCM_STREAM_MAGIC_ARRAY.length + 4; // magic_len + block_size_len

Review Comment:
   correct. But its length is always 4.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292845279


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;

Review Comment:
   I don't see why what I said wouldn't always be true. If so, we could eliminate this extra state.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292844750


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -53,29 +70,61 @@ public AesGcmEncryptor(byte[] keyBytes) {
       }
 
       this.randomGenerator = new SecureRandom();
+      this.nonce = new byte[NONCE_LENGTH];
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
-      byte[] nonce = new byte[NONCE_LENGTH];
-      randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;
       byte[] cipherText = new byte[cipherTextLength];
+      encrypt(plaintext, plaintextOffset, plaintextLength, cipherText, 0, aad);
+      return cipherText;
+    }
+
+    public int encrypt(
+        byte[] plaintext,
+        int plaintextOffset,
+        int plaintextLength,
+        byte[] ciphertextBuffer,
+        int ciphertextOffset,
+        byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
+      randomGenerator.nextBytes(nonce);
+      int enciphered;
 
       try {
         GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
         cipher.init(Cipher.ENCRYPT_MODE, aesKey, spec);
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
+        enciphered =
+            cipher.doFinal(
+                plaintext,
+                plaintextOffset,
+                plaintextLength,
+                ciphertextBuffer,
+                ciphertextOffset + NONCE_LENGTH);

Review Comment:
   I don't see where the `nonce` is copied into the output `cyphertextBuffer`. Is that done inside of `doFinal`? If so, can you add a comment that states what exactly `doFinal` puts into the output buffer?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1292844301


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -53,29 +70,61 @@ public AesGcmEncryptor(byte[] keyBytes) {
       }
 
       this.randomGenerator = new SecureRandom();
+      this.nonce = new byte[NONCE_LENGTH];
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
-      byte[] nonce = new byte[NONCE_LENGTH];
-      randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;
       byte[] cipherText = new byte[cipherTextLength];
+      encrypt(plaintext, plaintextOffset, plaintextLength, cipherText, 0, aad);
+      return cipherText;
+    }
+
+    public int encrypt(
+        byte[] plaintext,
+        int plaintextOffset,
+        int plaintextLength,
+        byte[] ciphertextBuffer,
+        int ciphertextOffset,
+        byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
+      randomGenerator.nextBytes(nonce);
+      int enciphered;
 
       try {
         GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
         cipher.init(Cipher.ENCRYPT_MODE, aesKey, spec);
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH);
+        enciphered =

Review Comment:
   Style: needs a newline before this.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r931813296


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+
+public class AesGcmInputFile implements InputFile {

Review Comment:
   `EncryptedInputFile` is an input to encryption manager's `decrypt` method; with the output being an `InputFile`, ready for direct consumption by readers. `AesGcmInputFile` is an output of the decrypt operation (its input_stream.read draws decrypted data); so it is given directly to manifest Avro readers, that use it in the usual manner (decryption is transparent to them).



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r931842123


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -96,33 +110,45 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
-      Preconditions.checkState(plainTextLength >= 1,
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad)  {
+      Preconditions.checkState(ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,
           "Cannot decrypt cipher text of length " + ciphertext.length +
           " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" +
           " with AES GCM cipher");
 
       // Get the nonce from ciphertext
       byte[] nonce = new byte[NONCE_LENGTH];
-      System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH);
+      System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH);
 
-      byte[] plainText = new byte[plainTextLength];
-      int inputLength = ciphertext.length - NONCE_LENGTH;
+      int inputLength = ciphertextLength - NONCE_LENGTH;
       try {
         GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce);
         cipher.init(Cipher.DECRYPT_MODE, aesKey, spec);
         if (null != aad) {
           cipher.updateAAD(aad);
         }
-        cipher.doFinal(ciphertext, NONCE_LENGTH, inputLength, plainText, 0);
+        return cipher.doFinal(ciphertext, ciphertextOffset + NONCE_LENGTH, inputLength);
       }  catch (AEADBadTagException e) {
         throw new RuntimeException("GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" +
-            "data. AES GCM doesn't differentiate between these two.. ", e);
+            "data. AES GCM doesn't differentiate between these two.", e);
       } catch (GeneralSecurityException e) {
         throw new RuntimeException("Failed to decrypt", e);
       }
+    }
+  }
 
-      return plainText;
+  static byte[] streamBlockAAD(byte[] fileAadPrefix, int currentBlockIndex) {
+    byte[] blockAAD = Ints.toByteArray(currentBlockIndex);

Review Comment:
   per the javadoc, https://github.com/google/guava/blob/master/guava/src/com/google/common/primitives/Ints.java#L300 ,
   this is an explicitly big endian operation. But we can also spell it out in a comment above this line in our code - or create our own ~ `intToBytesBigEndian` function. -?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r931834766


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -96,33 +110,45 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
-      Preconditions.checkState(plainTextLength >= 1,
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad)  {

Review Comment:
   sure, we can rename it to `decryptBlock`



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r931833985


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -53,10 +63,14 @@ public AesGcmEncryptor(byte[] keyBytes) {
       this.randomGenerator = new SecureRandom();
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {

Review Comment:
   Yep, this (and other GCM Stream fields) are not documented yet. In a way, the GCM Streams are a simple file format, designed to add encryption and integrity verification to existing Input/Output Streams, in a transparent (and efficient) manner. I agree we need an explicit spec for this. Should I add a .md file (to the `iceberg/format` folder) in this PR so we can discuss it here?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r929256455


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -19,19 +19,29 @@
 
 package org.apache.iceberg.encryption;
 
+import java.nio.charset.StandardCharsets;
 import java.security.GeneralSecurityException;
 import java.security.SecureRandom;
 import javax.crypto.AEADBadTagException;
 import javax.crypto.Cipher;
 import javax.crypto.spec.GCMParameterSpec;
 import javax.crypto.spec.SecretKeySpec;
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+import org.apache.iceberg.relocated.com.google.common.primitives.Ints;
 
 public class Ciphers {
-  private static final int NONCE_LENGTH = 12;
-  private static final int GCM_TAG_LENGTH = 16;
+  public static final int NONCE_LENGTH = 12;
+  public static final int GCM_TAG_LENGTH = 16;
+  public static final String GCM_STREAM_MAGIC_STRING = "GCM1";

Review Comment:
   Where does this come from? Is it introduced by Iceberg or from another standard?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r929254447


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+
+public class AesGcmInputFile implements InputFile {

Review Comment:
   Why not implement `EncryptedInputFile`?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r732111848


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+
+import static org.apache.iceberg.encryption.AesGcmOutputStream.HEADER_SIZE_LENGTH;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - HEADER_SIZE_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] blockSizeBytes = new byte[HEADER_SIZE_LENGTH];
+    int fetched = sourceStream.read(blockSizeBytes); // TODO check
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    plainBlockSize = ByteBuffer.wrap(blockSizeBytes).order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return (int) (plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    while (remaining > 0) {
+      sourceStream.seek(HEADER_SIZE_LENGTH + currentBlockIndex * cipherBlockSize);
+
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }

Review Comment:
   Use `Preconditions.checkArgument()` instead?



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+
+import static org.apache.iceberg.encryption.AesGcmOutputStream.HEADER_SIZE_LENGTH;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - HEADER_SIZE_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] blockSizeBytes = new byte[HEADER_SIZE_LENGTH];
+    int fetched = sourceStream.read(blockSizeBytes); // TODO check
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    plainBlockSize = ByteBuffer.wrap(blockSizeBytes).order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return (int) (plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }

Review Comment:
   Use `Preconditions.checkArgument(len > 0, "msg")` instead?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904551385


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);

Review Comment:
   oh, no, streams typically use long for these, eg
   ```java
   @Override
     public void seek(long newPos)
   ```



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904708479


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);

Review Comment:
   interesting though, the `available` doesn't have to be accurate, according to the InputStream contract. I presume this means that we don't need to use Math.toIntExact since it throws an exception if the result can't be mapped to an int. I'll change the logic to return max int in that case.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r903993495


##########
core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java:
##########
@@ -0,0 +1,104 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+
+import java.io.File;
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.util.Random;
+import org.apache.iceberg.Files;
+import org.apache.iceberg.io.PositionOutputStream;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.junit.Assert;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+public class TestGcmStreams {
+
+  @Rule
+  public TemporaryFolder temp = new TemporaryFolder();
+
+  @Test
+  public void testRandomWriteRead() throws IOException {

Review Comment:
   I like this, could I ask for just 4 more test cases
   
   Empty File (no bits)
   File that aligns perfectly with encryption Chunk Size
   File that is exactly one byte to larger than the aligned and one that is one byte smaller than the aligned file. (we probably hit this unaligned version with the testFileSize below but just to make sure)



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904097882


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);

Review Comment:
   Maybe i'm over thinking this, but shouldn't plainStreamSize and position both be ints? Just wondering why we need the safety check here



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902981810


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,201 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+
+import static org.apache.iceberg.encryption.AesGcmOutputStream.HEADER_SIZE_LENGTH;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;

Review Comment:
   Both plainStreamSize and fileAadPrefix could be final.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902970735


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputFile implements OutputFile {
+  private OutputFile targetFile;
+  private byte[] dataKey;
+
+  public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey) {
+    this.targetFile = targetFile;
+    this.dataKey = dataKey;
+  }
+
+  @Override
+  public PositionOutputStream create() {
+    try {
+      return new AesGcmOutputStream(targetFile.create(), dataKey, null);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);

Review Comment:
   Can we add message like, "Failed to create file: %s", targetFile.location()?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044708947


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,

Review Comment:
   This conclusion isn't correct. While unlikely to happen, the underlying stream is allowed to return fewer than the requested bytes. That isn't an error case and it doesn't indicate that the stream hit EOF.
   
   If you want to ensure that you get all the bytes requested, then use `IOUtil.readFully`.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278603093


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,274 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.EOFException;
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.IOUtil;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] cipherBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAADPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private byte[] currentDecryptedBlock;
+  private int currentDecryptedBlockIndex;
+
+  AesGcmInputStream(
+      SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix)
+      throws IOException {
+    long netSourceLength = netSourceFileLength(sourceLength);
+    boolean emptyCipherStream = (0 == netSourceLength);
+    this.sourceStream = sourceStream;
+    byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH];
+    IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length);
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(
+        Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string "
+            + Ciphers.GCM_STREAM_MAGIC_STRING);
+    this.currentDecryptedBlockIndex = -1;
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAADPrefix = fileAADPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize =
+          ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+              .order(ByteOrder.LITTLE_ENDIAN)
+              .getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      Preconditions.checkState(
+          plainBlockSize == AesGcmOutputStream.plainBlockSize,
+          "Wrong plainBlockSize "
+              + plainBlockSize
+              + ". Only size of "
+              + AesGcmOutputStream.plainBlockSize
+              + " is currently supported");
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.cipherBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize);
+      int cipherBytesInLastBlock =
+          Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize);
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      cipherBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAADPrefix = null;
+    }
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    Preconditions.checkState(len >= 0, "Negative read length " + len);
+
+    if (available() <= 0 && len > 0) {
+      throw new EOFException();
+    }
+
+    if (len == 0) {
+      return 0;
+    }
+
+    boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remainingBytesToRead = len;
+
+    while (remainingBytesToRead > 0) {
+      byte[] plainBlock = decryptNextBlock(isLastBlockInStream);
+
+      int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock;
+      int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead;
+      System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy);
+      remainingBytesToRead -= bytesToCopy;
+      resultBufferOffset += bytesToCopy;
+      currentOffsetInPlainBlock += bytesToCopy;
+
+      boolean endOfStream = isLastBlockInStream && finishTheBlock;
+
+      if (endOfStream) {
+        break;
+      }
+
+      if (finishTheBlock) {
+        currentBlockIndex++;
+        currentOffsetInPlainBlock = 0;
+        isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks);
+      }
+    }
+
+    plainStreamPosition += len - remainingBytesToRead;
+    return len - remainingBytesToRead;
+  }
+
+  @Override
+  public void seek(long newPos) throws IOException {
+    if (newPos < 0) {
+      throw new IOException("Negative new position " + newPos);
+    } else if (newPos > plainStreamSize) {
+      throw new EOFException(
+          "New position " + newPos + " exceeds the max stream size " + plainStreamSize);
+    }
+
+    currentBlockIndex = Math.toIntExact(newPos / plainBlockSize);
+    currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize);
+    plainStreamPosition = newPos;
+  }
+
+  @Override
+  public long skip(long n) {
+    if (n <= 0) {
+      return 0;
+    }
+
+    if (plainStreamPosition == plainStreamSize) {
+      return 0;
+    }
+
+    long newPosition = plainStreamPosition + n;
+
+    if (newPosition > plainStreamSize) {
+      long skipped = plainStreamSize - plainStreamPosition;
+      try {
+        seek(plainStreamSize);
+      } catch (IOException e) {
+        throw new UncheckedIOException(e);
+      }
+      return skipped;
+    }
+
+    try {
+      seek(newPosition);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+    return n;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return plainStreamPosition;
+  }
+
+  @Override
+  public int read() throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void close() throws IOException {
+    sourceStream.close();
+    currentDecryptedBlock = null;
+  }
+
+  static long calculatePlaintextLength(long sourceLength, int plainBlockSize) {
+    long netSourceFileLength = netSourceFileLength(sourceLength);
+
+    if (netSourceFileLength == 0) {
+      return 0;

Review Comment:
   I think this means that you can have a 0-length content file. That might make sense, but if we are going to allow a 0-length file then it should actually use a 0-length block as well.
   
   The problem with a 0-length stream is that an attacker can replace a file with a 0-length stream to effectively delete data while bypassing authentication checks. A 0-length block would prevent that attack because it would check the AAD for the block.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1279978125


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);

Review Comment:
   This should be done outside of the constructor before writing the first block.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1281477366


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -55,10 +71,15 @@ public AesGcmEncryptor(byte[] keyBytes) {
       this.randomGenerator = new SecureRandom();
     }
 
-    public byte[] encrypt(byte[] plainText, byte[] aad) {
+    public byte[] encrypt(byte[] plaintext, byte[] aad) {
+      return encrypt(plaintext, 0, plaintext.length, aad);
+    }
+
+    public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) {
+      Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength);
       byte[] nonce = new byte[NONCE_LENGTH];
       randomGenerator.nextBytes(nonce);
-      int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH;
+      int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH;

Review Comment:
   Done for simplicity, since manifest encryption won't benefit much from this optimization. 
   But we can add now, for Avro data encryption in the future.
   Btw, the same optimization can be performed for decryption.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1275496582


##########
.palantir/revapi.yml:
##########
@@ -743,6 +743,12 @@ acceptedBreaks:
       new: "method java.util.List<org.apache.iceberg.UpdateRequirement> org.apache.iceberg.rest.requests.UpdateTableRequest::requirements()"
       justification: "Signature changed to an interface, but this is safe because\
         \ of type erasure and the original type is always returned"
+  "1.3.0":
+    org.apache.iceberg:iceberg-core:
+    - code: "java.method.visibilityReduced"
+      old: "method void org.apache.iceberg.encryption.Ciphers::<init>()"
+      new: "method void org.apache.iceberg.encryption.Ciphers::<init>()"
+      justification: "Static utility class - should not have public constructor"

Review Comment:
   Is this needed? Can we just leave it as it was or is it now failing?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#issuecomment-1180891440

   Let's try to finalize this before we get 0.14/ 1.0


-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904531364


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");

Review Comment:
   Yep. We actually already have a Ciphers class; let's re-use it (with some additions) for all GCM-related things



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904345919


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");
+    this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+    this.currentBlockIndex = 0;
+    this.currentOffsetInPlainBlock = 0;
+
+    numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize);
+    lastBlockSize = (int) (netSourceFileSize % cipherBlockSize);
+    if (lastBlockSize == 0) {
+      lastBlockSize = cipherBlockSize;
+    } else {
+      numberOfBlocks += 1;
+    }
+
+    plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize +
+            (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH);
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    return Math.toIntExact(plainStreamSize - plainStreamPosition);
+  }
+
+  @Override
+  public int read(byte[] b) throws IOException {
+    return read(b, 0, b.length);
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len <= 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks;
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastBlockSize : cipherBlockSize;
+      int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad);
+      if (loaded != toLoad) {
+        throw new IOException("Read " + loaded + " instead of " + toLoad);
+      }
+
+      // Copy nonce
+      System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH);
+
+      byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex);
+      byte[] plaintextBlock;
+      try {
+        GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce);
+        gcmCipher.init(Cipher.DECRYPT_MODE, key, spec);
+        gcmCipher.updateAAD(aad);
+
+        plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH,
+                toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH);
+      } catch (GeneralSecurityException e) {
+        throw new IOException("Failed to decrypt", e);
+      }
+
+      int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock;
+      boolean finishTheBlock = remaining >= remainingInBlock;

Review Comment:
   Not sure I understand this? remainingInBlock is unencrypted bytes and remaining is encrypted right?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902971176


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java:
##########
@@ -0,0 +1,64 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputFile implements OutputFile {
+  private OutputFile targetFile;
+  private byte[] dataKey;
+
+  public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey) {
+    this.targetFile = targetFile;
+    this.dataKey = dataKey;
+  }
+
+  @Override
+  public PositionOutputStream create() {
+    try {
+      return new AesGcmOutputStream(targetFile.create(), dataKey, null);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);
+    }
+  }
+
+  @Override
+  public PositionOutputStream createOrOverwrite() {
+    try {
+      return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, null);
+    } catch (IOException e) {
+      throw new UncheckedIOException(e);

Review Comment:
   Can we add message like, "Failed to create or overwrite file: %s", targetFile.location()?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902982423


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;

Review Comment:
   Can we rename it to lastCipherBlockSize to make it more readable?



##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;

Review Comment:
   final as well.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] flyrain commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
flyrain commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r902981199


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);

Review Comment:
   They can be in the same line.
   IIUC, this means the stream length is not long enough. Can we say something like "The stream length should be at least " + AesGcmOutputStream.PREFIX_LENGTH?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] RussellSpitzer commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r904075687


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,212 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.security.GeneralSecurityException;
+import java.util.Arrays;
+import javax.crypto.Cipher;
+import javax.crypto.SecretKey;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private SeekableInputStream sourceStream;
+  private long netSourceFileSize;
+
+  private Cipher gcmCipher;
+  private SecretKey key;
+  private byte[] nonce;
+
+  private byte[] ciphertextBlockBuffer;
+  private int cipherBlockSize;
+  private int plainBlockSize;
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+  private int numberOfBlocks;
+  private int lastBlockSize;
+  private long plainStreamSize;
+  private byte[] fileAadPrefix;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH;
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH,
+        "Insufficient read " + fetched);
+    this.plainStreamPosition = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length);
+
+    Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic),
+        "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING);
+
+    plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4)
+        .order(ByteOrder.LITTLE_ENDIAN).getInt();
+    cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH;
+
+    try {
+      gcmCipher = Cipher.getInstance("AES/GCM/NoPadding");
+    } catch (GeneralSecurityException e) {
+      throw new IOException(e);
+    }
+    this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH];
+    this.key = new SecretKeySpec(aesKey, "AES");

Review Comment:
   Constant?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1054069247


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");

Review Comment:
   Sounds reasonable, I'll apply this to the code.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1054076086


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(Ciphers.GCM_STREAM_PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastCipherBlockSize : cipherBlockSize;

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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1039893751


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java:
##########
@@ -0,0 +1,71 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.SeekableInputStream;
+
+public class AesGcmInputFile implements InputFile {
+  private final InputFile sourceFile;
+  private final byte[] dataKey;
+  private long plaintextLength;
+
+  public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) {
+    this.sourceFile = sourceFile;
+    this.dataKey = dataKey;
+    this.plaintextLength = -1;
+  }
+
+  @Override
+  public long getLength() {
+    if (plaintextLength == -1) {
+      try {
+        this.newStream().close();

Review Comment:
   We should be able to know the length without opening the stream, right?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1039912960


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];

Review Comment:
   Can we call this a header? We use `prefix` in several places and I think it would be more clear to call it a header since it is only at the start of the file.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1039939654


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remaining = len;

Review Comment:
   Using `remaining` is a bit confusing because its meaning isn't clear. It could be remaining in the encrypted stream or remaining bytes to read, or something else. I'd recommend being more specific and using `bytesToRead`.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1039945676


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");

Review Comment:
   If the unencrypted file length is 0, we should return an in-memory input stream with 0 bytes instead of a decrypting one.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by GitBox <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1044728554


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java:
##########
@@ -0,0 +1,218 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import java.util.Arrays;
+import org.apache.iceberg.io.SeekableInputStream;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+public class AesGcmInputStream extends SeekableInputStream {
+  private final SeekableInputStream sourceStream;
+  private final boolean emptyCipherStream;
+  private final long netSourceFileSize;
+  private final Ciphers.AesGcmDecryptor gcmDecryptor;
+  private final byte[] ciphertextBlockBuffer;
+  private final int cipherBlockSize;
+  private final int plainBlockSize;
+  private final int numberOfBlocks;
+  private final int lastCipherBlockSize;
+  private final long plainStreamSize;
+  private final byte[] fileAadPrefix;
+
+  private long plainStreamPosition;
+  private int currentBlockIndex;
+  private int currentOffsetInPlainBlock;
+
+  AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength,
+                    byte[] aesKey, byte[] fileAadPrefix) throws IOException {
+    this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH;
+    Preconditions.checkArgument(netSourceFileSize >= 0,
+        "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted");
+
+    this.emptyCipherStream = (0 == netSourceFileSize);
+    this.sourceStream = sourceStream;
+    byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH];
+    int fetched = sourceStream.read(prefixBytes);
+    Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH,
+        "Insufficient read " + fetched +
+            ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH);
+
+    byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length];
+    System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length);
+    Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic),
+        "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING);
+
+    if (!emptyCipherStream) {
+      this.plainStreamPosition = 0;
+      this.fileAadPrefix = fileAadPrefix;
+      gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey);
+      plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4)
+          .order(ByteOrder.LITTLE_ENDIAN).getInt();
+      Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize);
+
+      cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
+      this.ciphertextBlockBuffer = new byte[cipherBlockSize];
+      this.currentBlockIndex = 0;
+      this.currentOffsetInPlainBlock = 0;
+
+      int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize);
+      int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize);
+      boolean fullBlocksOnly = (0 == cipherBytesInLastBlock);
+      numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1;
+      lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0
+      int plainBytesInLastBlock = fullBlocksOnly ? 0 :
+          (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH);
+      plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock;
+    } else {
+      plainStreamSize = 0;
+
+      gcmDecryptor = null;
+      ciphertextBlockBuffer = null;
+      cipherBlockSize = -1;
+      plainBlockSize = -1;
+      numberOfBlocks = -1;
+      lastCipherBlockSize = -1;
+      this.fileAadPrefix = null;
+    }
+  }
+
+  public long plaintextStreamSize() {
+    return plainStreamSize;
+  }
+
+  @Override
+  public int available() throws IOException {
+    long maxAvailable = plainStreamSize - plainStreamPosition;
+    // See InputStream.available contract
+    if (maxAvailable >= Integer.MAX_VALUE) {
+      return Integer.MAX_VALUE;
+    } else {
+      return (int) maxAvailable;
+    }
+  }
+
+  @Override
+  public int read(byte[] b, int off, int len) throws IOException {
+    if (len < 0) {
+      throw new IOException("Negative read length " + len);
+    }
+
+    if (available() <= 0) {
+      return -1;
+    }
+
+    boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks);
+    int resultBufferOffset = off;
+    int remaining = len;
+
+    sourceStream.seek(Ciphers.GCM_STREAM_PREFIX_LENGTH + currentBlockIndex * cipherBlockSize);
+
+    while (remaining > 0) {
+      int toLoad = lastBlock ? lastCipherBlockSize : cipherBlockSize;

Review Comment:
   I find the naming a bit confusing because it is inconsistent. I think it is good to use `plain` and `cipher` to distinguish but we should do that consistently. In this case, I would call this `currentCipherBlockSize`.
   
   I think it would also be good to break this up so you're not combining encrypted block reads and decrypts with the logic to copy data from a plaintext buffer. Can you add a `nextCipherBlock` or `nextPlainBlock` method that handles this?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] ggershinsky commented on a diff in pull request #3231: GCM encryption stream

Posted by "ggershinsky (via GitHub)" <gi...@apache.org>.
ggershinsky commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1293167115


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {

Review Comment:
   I've added a check making sure the block is not empty



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1315166849


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java:
##########
@@ -0,0 +1,115 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.nio.ByteBuffer;
+import java.nio.ByteOrder;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputStream extends PositionOutputStream {
+
+  private final Ciphers.AesGcmEncryptor gcmEncryptor;
+  private final PositionOutputStream targetStream;
+  private final byte[] plainBlockBuffer;
+  private final byte[] fileAadPrefix;
+
+  private int positionInBuffer;
+  private long streamPosition;
+  private int currentBlockIndex;
+
+  AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix)
+      throws IOException {
+    this.targetStream = targetStream;
+    this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey);
+    this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE];
+    this.positionInBuffer = 0;
+    this.streamPosition = 0;
+    this.currentBlockIndex = 0;
+    this.fileAadPrefix = fileAadPrefix;
+
+    byte[] headerBytes =
+        ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH)
+            .order(ByteOrder.LITTLE_ENDIAN)
+            .put(Ciphers.GCM_STREAM_MAGIC_ARRAY)
+            .putInt(Ciphers.PLAIN_BLOCK_SIZE)
+            .array();
+    targetStream.write(headerBytes);
+  }
+
+  @Override
+  public void write(int b) throws IOException {
+    throw new UnsupportedOperationException();
+  }
+
+  @Override
+  public void write(byte[] b, int off, int len) throws IOException {
+    if (b.length - off < len) {
+      throw new IOException(
+          "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len);
+    }
+    int remaining = len;
+    int offset = off;
+
+    while (remaining > 0) {
+      int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer;
+      int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining;
+
+      System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite);
+      positionInBuffer += toWrite;
+      if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) {
+        encryptAndWriteBlock();
+        positionInBuffer = 0;
+      }
+      offset += toWrite;
+      remaining -= toWrite;
+    }
+
+    streamPosition += len;
+  }
+
+  @Override
+  public long getPos() throws IOException {
+    return streamPosition;
+  }
+
+  @Override
+  public void flush() throws IOException {
+    targetStream.flush();
+  }
+
+  @Override
+  public void close() throws IOException {
+    if (positionInBuffer > 0) {
+      encryptAndWriteBlock();
+    }
+    targetStream.close();
+  }
+
+  private void encryptAndWriteBlock() throws IOException {

Review Comment:
   Actually, I meant that it would be good to have a check that only one partial block (last block) is written.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278601221


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -101,37 +117,52 @@ public AesGcmDecryptor(byte[] keyBytes) {
     }
 
     public byte[] decrypt(byte[] ciphertext, byte[] aad) {
-      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      return decrypt(ciphertext, 0, ciphertext.length, aad);
+    }
+
+    public byte[] decrypt(
+        byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) {
       Preconditions.checkState(
-          plainTextLength >= 1,
+          ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1,

Review Comment:
   Should there be a similar check in the encrypt method to validate that the plaintext has at least one byte?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1278598984


##########
core/src/main/java/org/apache/iceberg/encryption/Ciphers.java:
##########
@@ -27,10 +30,19 @@
 import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
 
 public class Ciphers {
-  private static final int NONCE_LENGTH = 12;
-  private static final int GCM_TAG_LENGTH = 16;
+  public static final int NONCE_LENGTH = 12;
+  public static final int GCM_TAG_LENGTH = 16;
+  public static final String GCM_STREAM_MAGIC_STRING = "AGS1";
+
+  static final byte[] GCM_STREAM_MAGIC_ARRAY =
+      GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8);
+  static final int GCM_STREAM_HEADER_LENGTH =
+      GCM_STREAM_MAGIC_ARRAY.length + 4; // magic_len + block_size_len

Review Comment:
   I thought we had decided on a static block size?



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org


[GitHub] [iceberg] rdblue commented on a diff in pull request #3231: GCM encryption stream

Posted by "rdblue (via GitHub)" <gi...@apache.org>.
rdblue commented on code in PR #3231:
URL: https://github.com/apache/iceberg/pull/3231#discussion_r1279976921


##########
core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java:
##########
@@ -0,0 +1,66 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.iceberg.encryption;
+
+import java.io.IOException;
+import java.io.UncheckedIOException;
+import org.apache.iceberg.io.InputFile;
+import org.apache.iceberg.io.OutputFile;
+import org.apache.iceberg.io.PositionOutputStream;
+
+public class AesGcmOutputFile implements OutputFile {
+  private final OutputFile targetFile;
+  private final byte[] dataKey;
+  private final byte[] fileAADPrefix;
+
+  public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey, byte[] fileAADPrefix) {
+    this.targetFile = targetFile;
+    this.dataKey = dataKey;
+    this.fileAADPrefix = fileAADPrefix;
+  }
+
+  @Override
+  public PositionOutputStream create() {
+    try {
+      return new AesGcmOutputStream(targetFile.create(), dataKey, fileAADPrefix);
+    } catch (IOException e) {
+      throw new UncheckedIOException("Failed to create GCM stream for " + targetFile.location(), e);
+    }
+  }
+
+  @Override
+  public PositionOutputStream createOrOverwrite() {
+    try {
+      return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, fileAADPrefix);
+    } catch (IOException e) {
+      throw new UncheckedIOException(
+          "Failed to create or overwrite GCM stream for " + targetFile.location(), e);
+    }
+  }
+
+  @Override
+  public String location() {
+    return targetFile.location();
+  }
+
+  @Override
+  public InputFile toInputFile() {
+    throw new UnsupportedOperationException();

Review Comment:
   This should be implemented. I think it should be fairly easy since this has the same fields.



-- 
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: issues-unsubscribe@iceberg.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org


---------------------------------------------------------------------
To unsubscribe, e-mail: issues-unsubscribe@iceberg.apache.org
For additional commands, e-mail: issues-help@iceberg.apache.org