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 2021/05/26 13:11:50 UTC

[GitHub] [iceberg] ggershinsky opened a new pull request #2638: Core: Native encryption params

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


   This is a PR based on the design doc https://docs.google.com/document/d/1kkcjr9KrlB9QagRX3ToulG_Rf-65NMSlVANheDNzJq4/edit#
   
   It introduces a new set of encryption APIs/parameters for the native encryption formats (Parquet and ORC). 
   The basic parameters are the column keys for a file. 
   In addition, it passes a file/footer key and an AADPrefix, that are consumed by Parquet but can be ignored by ORC.
   
   @rdblue @jackye1995 @RussellSpitzer @flyrain @aokolnychyi 


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

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] jackye1995 commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826370856



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import javax.crypto.AEADBadTagException;
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {

Review comment:
       can use `Preconditions.checkArgument`




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r831945258



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);
+      }
+
+      // Add the nonce
+      System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH);
+
+      return cipherText;
+    }
+  }
+
+  public static class AesGcmDecryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+
+    public AesGcmDecryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+    }
+
+    public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
+      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      Preconditions.checkState(plainTextLength >= 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);
+
+      byte[] plainText = new byte[plainTextLength];
+      int inputLength = ciphertext.length - 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);
+      }  catch (AEADBadTagException e) {
+        throw new RuntimeException("GCM tag check failed", e);

Review comment:
       agreed, this one is tricky. I'll add the info.




-- 
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 #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-1051320036


   We've done a bunch of review on this internally, but would like to get some more feedback before we merge it. @jackye1995 Since you are the other main architect of this, could you take a look?


-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r793168792



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * a minimum client interface to connect to a key management service (KMS).
+ */
+public interface NativelyEncryptedFile {

Review comment:
       Should we move it to module `api` instead of `core`?




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r793309107



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * a minimum client interface to connect to a key management service (KMS).
+ */
+public interface NativelyEncryptedFile {

Review comment:
       It depends on the `NativeFileCryptoParameters` class, located in `core`




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r831940475



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);

Review comment:
       Mm, the callers won't have a type-specific reaction to this. The text itself provides the info for a post-mortem analysis. So probably this goes down to Iceberg's coding practices. 




-- 
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 #2638: Core: Native encryption params

Posted by GitBox <gi...@apache.org>.
rdblue commented on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-860128179


   I'm also skeptical about adding a "native" encryption API. It isn't clear what that is from the name and I think that using Parquet and ORC encryption is expected for those files. I would like to make the transition to configuring Parquet and ORC encryption a bit more seamless with a reasonable fallback if you want to use stream encryption for those formats.


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

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 edited a comment on pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky edited a comment on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-1027658982


   Done.  It was a CI glitch; I've re-opened the PR, all tests pass.


-- 
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] jackye1995 commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826373617



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileKey;
+  private String fileEncryptionAlgorithm;
+
+  private NativeFileCryptoParameters(ByteBuffer fileKey, String fileEncryptionAlgorithm) {
+    Preconditions.checkState(fileKey != null, "File encryption key is not supplied");
+    this.fileKey = fileKey;
+    this.fileEncryptionAlgorithm = fileEncryptionAlgorithm;
+  }
+
+  /**
+   * Creates the builder.
+   *
+   * @param fileKey per-file encryption key. For example, used as "footer key" DEK in Parquet encryption.
+   */
+  public static Builder create(ByteBuffer fileKey) {
+    return new Builder(fileKey);
+  }
+
+  public static class Builder {
+    private ByteBuffer fileKey;
+    private String fileEncryptionAlgorithm;
+
+    private Builder(ByteBuffer fileKey) {
+      this.fileKey = fileKey;
+    }
+
+    public Builder encryptionAlgorithm(String encryptionAlgorithm) {
+      this.fileEncryptionAlgorithm = encryptionAlgorithm;
+      return this;
+    }
+
+    public NativeFileCryptoParameters build() {
+      return new NativeFileCryptoParameters(fileKey, fileEncryptionAlgorithm);
+    }
+
+    // TODO add back column encryption keys

Review comment:
       nit: these are new features and we already mentioned these in the top class javadoc, can we remove the TODOs?




-- 
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] jackye1995 commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826388350



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileKey;
+  private String fileEncryptionAlgorithm;

Review comment:
       why not use the enum EncryptionAlgorithm?




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r831934899



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;

Review comment:
       sure, why not




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r793166055



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import javax.crypto.AEADBadTagException;
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+public class Ciphers {
+
+  public static class AesGcmEncryptor {
+    public static final int NONCE_LENGTH = 12;
+    public static final int GCM_TAG_LENGTH = 16;
+    public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Wrong key length " + keyLength);
+      }
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);
+      }
+
+      // Add the nonce
+      System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH);
+
+      return cipherText;
+    }
+  }
+
+  public static class AesGcmDecryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+
+    public AesGcmDecryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Wrong key length " + keyLength);

Review comment:
       We need the same error message as line 42.




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826763368



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import javax.crypto.AEADBadTagException;
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Cannot use a key of length " + keyLength +
+            " because AES only allows 16, 24 or 32 bytes");
+      }
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);
+      }
+
+      // Add the nonce
+      System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH);
+
+      return cipherText;
+    }
+  }
+
+  public static class AesGcmDecryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+
+    public AesGcmDecryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Cannot use a key of length " + keyLength +
+            " because AES only allows 16, 24 or 32 bytes");
+      }
+
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+    }
+
+    public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
+      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      if (plainTextLength < 1) {
+        throw new RuntimeException("Cannot decrypt cipher text of length " + ciphertext.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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826762783



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import javax.crypto.AEADBadTagException;
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {

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] jackye1995 commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826373617



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileKey;
+  private String fileEncryptionAlgorithm;
+
+  private NativeFileCryptoParameters(ByteBuffer fileKey, String fileEncryptionAlgorithm) {
+    Preconditions.checkState(fileKey != null, "File encryption key is not supplied");
+    this.fileKey = fileKey;
+    this.fileEncryptionAlgorithm = fileEncryptionAlgorithm;
+  }
+
+  /**
+   * Creates the builder.
+   *
+   * @param fileKey per-file encryption key. For example, used as "footer key" DEK in Parquet encryption.
+   */
+  public static Builder create(ByteBuffer fileKey) {
+    return new Builder(fileKey);
+  }
+
+  public static class Builder {
+    private ByteBuffer fileKey;
+    private String fileEncryptionAlgorithm;
+
+    private Builder(ByteBuffer fileKey) {
+      this.fileKey = fileKey;
+    }
+
+    public Builder encryptionAlgorithm(String encryptionAlgorithm) {
+      this.fileEncryptionAlgorithm = encryptionAlgorithm;
+      return this;
+    }
+
+    public NativeFileCryptoParameters build() {
+      return new NativeFileCryptoParameters(fileKey, fileEncryptionAlgorithm);
+    }
+
+    // TODO add back column encryption keys

Review comment:
       nit: these are new features, can we remove the TODOs?




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826773600



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {

Review comment:
       by "carries the file encryption key", we mean in-memory-only delivery of the crypto parameters from the Iceberg key management module, to the format-native writer/reader objects. The key management is distributed, but it is handled by the EncryptionManager interface that already extends `Serializable`; therefore, we have a local representation, an EncryptionManager object co-located with the native format objects in the same process memory.




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r788677534



##########
File path: api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java
##########
@@ -37,8 +39,15 @@
   OutputFile encryptingOutputFile();
 
   /**
-   * Metadata about the encryption key that is being used to encrypt the associated
+   * Metadata about the encryption keys and other crypto parameters used to encrypt the associated
    * {@link #encryptingOutputFile()}.
    */
   EncryptionKeyMetadata keyMetadata();
+
+  /**
+   * Parameters of native encryption (if used for this file)
+   */
+  default NativeFileCryptoParameters nativeEncryptionParameters() {

Review comment:
       Not quite; all encrypted files (inc natively encrypted) have `key_metadata`, since it is required to decrypt them. But, as I mentioned in the previous comment, I'll play with applying a new interface to the output too, so maybe the `EncryptedOutputFile` doesn't need to 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] jackye1995 commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826371400



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import javax.crypto.AEADBadTagException;
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Cannot use a key of length " + keyLength +
+            " because AES only allows 16, 24 or 32 bytes");
+      }
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);
+      }
+
+      // Add the nonce
+      System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH);
+
+      return cipherText;
+    }
+  }
+
+  public static class AesGcmDecryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+
+    public AesGcmDecryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {

Review comment:
       can use `Preconditions.checkArgument`




-- 
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] jackye1995 commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826386745



##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java
##########
@@ -20,12 +20,16 @@
 package org.apache.iceberg.aws.s3;
 
 import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.encryption.NativeFileCryptoParameters;
+import org.apache.iceberg.encryption.NativelyEncryptedFile;
 import org.apache.iceberg.io.InputFile;
 import org.apache.iceberg.io.SeekableInputStream;
 import org.apache.iceberg.metrics.MetricsContext;
 import software.amazon.awssdk.services.s3.S3Client;
 
-public class S3InputFile extends BaseS3File implements InputFile {
+public class S3InputFile extends BaseS3File implements InputFile, NativelyEncryptedFile {
+  private NativeFileCryptoParameters nativeDecryptionParameters;

Review comment:
       can you make the the parameter name match the class name?




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r788081044



##########
File path: api/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (and optional AAD prefix, column keys).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileAadPrefix;
+  private Map<String, ByteBuffer> columnKeys;
+  private ByteBuffer fileKey;
+  private String fileEncryptionAlgorithm;
+
+  private NativeFileCryptoParameters(Map<String, ByteBuffer> columnKeys, ByteBuffer fileKey,
+                                     ByteBuffer fileAadPrefix, String fileEncryptionAlgorithm) {
+    Preconditions.checkState((columnKeys != null && columnKeys.size() > 0) || fileKey != null,
+            "No file or column keys are supplied");
+    this.columnKeys = columnKeys;
+    this.fileKey = fileKey;
+    this.fileAadPrefix = fileAadPrefix;
+    this.fileEncryptionAlgorithm = fileEncryptionAlgorithm;
+  }
+
+  /**
+   * Creates the builder.

Review comment:
       Nit: we normally have an empty line between line 48 and 49.




-- 
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 #2638: Core: Encryption basics

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


   It was a CI glitch. All tests pass.


-- 
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 #2638: Core: Encryption basics

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


   hi @jackye1995 , thanks for the review. I've sent a commit that addresses 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] jackye1995 commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826372659



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,130 @@
+/*
+ * 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.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import javax.crypto.AEADBadTagException;
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Cannot use a key of length " + keyLength +
+            " because AES only allows 16, 24 or 32 bytes");
+      }
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);
+      }
+
+      // Add the nonce
+      System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH);
+
+      return cipherText;
+    }
+  }
+
+  public static class AesGcmDecryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+
+    public AesGcmDecryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Cannot use a key of length " + keyLength +
+            " because AES only allows 16, 24 or 32 bytes");
+      }
+
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+    }
+
+    public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
+      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      if (plainTextLength < 1) {
+        throw new RuntimeException("Cannot decrypt cipher text of length " + ciphertext.length +

Review comment:
       should this be an `IllegalStateException` to be more specific? (with `Preconditions.checkState`)




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r831947411



##########
File path: core/src/main/java/org/apache/iceberg/encryption/EncryptionAlgorithm.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+/**
+ * Algorithm supported for file encryption.
+ */
+public enum EncryptionAlgorithm {
+    /**
+     * Counter mode (CTR) allows fast encryption with high throughput.
+     * It is an encryption only cipher and does not ensure content integrity.
+     * Inputs to CTR cipher are:
+     *  1. encryption key
+     *  2. a 16-byte initialization vector (12-byte nonce, 4-byte counter)
+     *  3. plaintext data
+     */
+    AES_CTR,
+    /**
+     * Galois/Counter mode (GCM) combines CTR with the new Galois mode of authentication.
+     * It not only ensures data confidentiality, but also ensures data integrity.
+     * Inputs to GCM cipher are:
+     *  1. encryption key
+     *  2. a 12-byte initialization vector
+     *  3. additional authenticated data
+     *  4. plaintext data
+     */
+    AES_GCM,
+    /**
+     * A combination of GCM and CTR that can be used for file types like Parquet,
+     * so that all modules except pages are encrypted by GCM to ensure integrity,
+     * and CTR is used for efficient encryption of bulk data.
+     * The tradeoff is that attackers would be able to tamper page data.

Review comment:
       above, we have "CTR is used for .. encryption of .. data". But I'll update the 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] ggershinsky commented on pull request #2638: Core: Native encryption params

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


   Sure, TBD, but we need to sync up first on this point
   > there are enough information provided in key_metadata.
   
   which (in my view) seems to be a major gap here - `key_metadata` is a field in manifest files, which keeps the wrapped encryption keys for each data file. The keys (DEKs) are either generated randomly by Iceberg, or fetched from a KMS. The key wrapping is always done via KMS calls. Key_metadata doesn't store the keys themselves, since this would make them exposed in manifest files. Therefore, passing `key_metadata` to the data file writers doesn't help, because these writers need the data encryption keys. Calling KMS to unwrap them doesn't make sense in this particular point... Therefore, we need a separate mechanism to deliver the DEK(s) and other parameters to the writers. The classes in this PR list the required parameters. 
   We have a similar issue on the reader side, but lets get to it after syncing on the writer path.


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

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 pull request #2638: Core: Native encryption params

Posted by GitBox <gi...@apache.org>.
flyrain commented on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-852637895


   Hi @aokolnychyi, can you approve the workflows for this 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.

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 edited a comment on pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky edited a comment on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-1048075483


   > Can you please fix the conflicts?
   
   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] RussellSpitzer commented on pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
RussellSpitzer commented on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-1048007629


   @ggershinsky Can you please fix the conflicts?
   
   @jackye1995 could you take another pass on this? I've looked it over internally and I think we should be good to go here. I'd like to start making progress merging some of these in.


-- 
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 #2638: Core: Encryption basics

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


   > Can you please fix the conflicts?
   
   Sure, will 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] flyrain commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r788084411



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import javax.crypto.AEADBadTagException;
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+public class Ciphers {
+
+  public static class AesGcmEncryptor {
+    public static final int NONCE_LENGTH = 12;
+    public static final int GCM_TAG_LENGTH = 16;
+    public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Wrong key length " + keyLength);
+      }
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);
+      }
+
+      // Add the nonce
+      System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH);
+
+      return cipherText;
+    }
+  }
+
+  public static class AesGcmDecryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+
+    public AesGcmDecryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Wrong key length " + keyLength);
+      }
+
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+    }
+
+    public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
+      int plainTextLength = ciphertext.length - AesGcmEncryptor.GCM_TAG_LENGTH - AesGcmEncryptor.NONCE_LENGTH;

Review comment:
       Nit: How about moving `AesGcmEncryptor.GCM_TAG_LENGTH`, `AesGcmEncryptor.NONCE_LENGTH`, `AesGcmEncryptor.GCM_TAG_LENGTH_BITS` under the class `Ciper`? So that class `AesGcmDecryptor` doesn't invoke the static methods in class `AesGcmEncryptor`.




-- 
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] jackye1995 commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826386047



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {

Review comment:
       I think this needs to implement `Serializable`




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r794076621



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * a minimum client interface to connect to a key management service (KMS).
+ */
+public interface NativelyEncryptedFile {

Review comment:
       We will need an interface for NativeFileCryptoParameters in that case, which should located in module `API`.




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r793168792



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * a minimum client interface to connect to a key management service (KMS).
+ */
+public interface NativelyEncryptedFile {

Review comment:
       Should we move it to module `api` instead of putting it in `core`? It also requires to create an interface for `NativeFileCryptoParameters`.  Not sure that's a better way. Would like to hear people's thoughts. cc @RussellSpitzer 




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826773600



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {

Review comment:
       by "carries the file encryption key", we mean in-memory-only delivery of the crypto parameters from the Iceberg key management module, to the format-native writer/reader objects. The key management is distributed, but it is handled by the EncryptionManager interface that already extends `Serialize`; therefore, we have a local representation, an EncryptionManager object co-located with the native format objects in the same process memory.




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826763733



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileKey;
+  private String fileEncryptionAlgorithm;
+
+  private NativeFileCryptoParameters(ByteBuffer fileKey, String fileEncryptionAlgorithm) {
+    Preconditions.checkState(fileKey != null, "File encryption key is not supplied");
+    this.fileKey = fileKey;
+    this.fileEncryptionAlgorithm = fileEncryptionAlgorithm;
+  }
+
+  /**
+   * Creates the builder.
+   *
+   * @param fileKey per-file encryption key. For example, used as "footer key" DEK in Parquet encryption.
+   */
+  public static Builder create(ByteBuffer fileKey) {
+    return new Builder(fileKey);
+  }
+
+  public static class Builder {
+    private ByteBuffer fileKey;
+    private String fileEncryptionAlgorithm;
+
+    private Builder(ByteBuffer fileKey) {
+      this.fileKey = fileKey;
+    }
+
+    public Builder encryptionAlgorithm(String encryptionAlgorithm) {
+      this.fileEncryptionAlgorithm = encryptionAlgorithm;
+      return this;
+    }
+
+    public NativeFileCryptoParameters build() {
+      return new NativeFileCryptoParameters(fileKey, fileEncryptionAlgorithm);
+    }
+
+    // TODO add back column encryption keys

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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826791338



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,78 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileKey;
+  private String fileEncryptionAlgorithm;

Review comment:
       a good point. I was thinking of making the native parameters totally bottom-up (Parquet/ORC) and independent of Iceberg parameters. But it makes sense to start the mapping here and re-use the `EncryptionAlgorithm` enum.




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r788081197



##########
File path: api/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,106 @@
+/*
+ * 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.nio.ByteBuffer;
+import java.util.Map;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (and optional AAD prefix, column keys).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileAadPrefix;
+  private Map<String, ByteBuffer> columnKeys;
+  private ByteBuffer fileKey;
+  private String fileEncryptionAlgorithm;
+
+  private NativeFileCryptoParameters(Map<String, ByteBuffer> columnKeys, ByteBuffer fileKey,
+                                     ByteBuffer fileAadPrefix, String fileEncryptionAlgorithm) {
+    Preconditions.checkState((columnKeys != null && columnKeys.size() > 0) || fileKey != null,
+            "No file or column keys are supplied");
+    this.columnKeys = columnKeys;
+    this.fileKey = fileKey;
+    this.fileAadPrefix = fileAadPrefix;
+    this.fileEncryptionAlgorithm = fileEncryptionAlgorithm;
+  }
+
+  /**
+   * Creates the builder.
+   * @param fileKey per-file encryption key. For example, used as "footer key" DEK in Parquet encryption.
+   */
+  public static Builder create(ByteBuffer fileKey) {
+    return new Builder(fileKey);
+  }
+
+  public static class Builder {
+    private ByteBuffer fileAadPrefix;
+    private Map<String, ByteBuffer> columnKeys;
+    private ByteBuffer fileKey;
+    private String fileEncryptionAlgorithm;
+
+    private Builder(ByteBuffer fileKey) {
+      this.fileKey = fileKey;
+    }
+
+    /**
+     * Set column encryption keys.

Review comment:
       Nit: Empty line here 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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r788083683



##########
File path: api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java
##########
@@ -37,8 +39,15 @@
   OutputFile encryptingOutputFile();
 
   /**
-   * Metadata about the encryption key that is being used to encrypt the associated
+   * Metadata about the encryption keys and other crypto parameters used to encrypt the associated
    * {@link #encryptingOutputFile()}.
    */
   EncryptionKeyMetadata keyMetadata();
+
+  /**
+   * Parameters of native encryption (if used for this file)
+   */
+  default NativeFileCryptoParameters nativeEncryptionParameters() {

Review comment:
       Can we reuse `keyMetadata ` instead of introducing `nativeEncryptionParameters `? Here are reasons:
   1. Both `keyMetadata` and `nativeEncryptionParameters` are used for keys and crypto parameters.
   2. Each file will have either one, not both. It depends whether streaming encryption or native encryption is used for this file.
   
   NativeFileCryptoParameters can implement interface `EncryptionKeyMetadata` in that sense.




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r788084805



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,127 @@
+/*
+ * 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.security.GeneralSecurityException;
+import java.security.SecureRandom;
+import javax.crypto.AEADBadTagException;
+import javax.crypto.Cipher;
+import javax.crypto.spec.GCMParameterSpec;
+import javax.crypto.spec.SecretKeySpec;
+
+public class Ciphers {
+
+  public static class AesGcmEncryptor {
+    public static final int NONCE_LENGTH = 12;
+    public static final int GCM_TAG_LENGTH = 16;
+    public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Wrong key length " + keyLength);
+      }
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);
+      }
+
+      // Add the nonce
+      System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH);
+
+      return cipherText;
+    }
+  }
+
+  public static class AesGcmDecryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+
+    public AesGcmDecryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      if (!(keyLength == 16 || keyLength == 24 || keyLength == 32)) {
+        throw new IllegalArgumentException("Wrong key length " + keyLength);

Review comment:
       Suggested to add the error message that key length can only be 16/24/32.




-- 
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 #2638: Core: Native encryption params

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


   Yep, this PR is not a public API or anything visible to the user. It is a fully internal set of classes, that carry the barebone parameters required to activate Parquet or ORC encryption (the only formats with native encryption today). These parameters are mostly the column encryption keys; plus the footer key required for Parquet (will be ignored by ORC). Plus the AAD prefix - also Parquet-specific, but can be left for a future version, when we're done with basic table  encryption, and start working on end-to-end table integrity protection.
   
   (regarding the public API, I also think there are situations where a user might want to explicitly chose either "native" Parquet/ORC column encryption - or flat stream encryption for same Parquet/ORC files. But this is likely a future/advanced option, and in any case, out of scope for this 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.

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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r793168792



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * a minimum client interface to connect to a key management service (KMS).
+ */
+public interface NativelyEncryptedFile {

Review comment:
       Should we move it to module `api` instead of putting it in `core`? It also requires to create an interface for `NativeFileCryptoParameters`.  Not sure that's a better way. Love to hear people's thoughts. cc @RussellSpitzer 




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r788674371



##########
File path: api/src/main/java/org/apache/iceberg/io/InputFile.java
##########
@@ -59,4 +60,18 @@
    * @return true if the file exists, false otherwise
    */
   boolean exists();
+
+  // TODO remove this comment after review

Review comment:
       I like this idea. Regarding the details - using `EncryptedInputFile` directly won't work (because this is a producer/decryptor of input files; an object shouldn't be both an input file and its decryptor) - but we can add a new interface, something like `NativeEncryption`, and make the classes implement it. Moreover, we can apply this to output files, not only to input files. I'll play with 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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r826783675



##########
File path: aws/src/main/java/org/apache/iceberg/aws/s3/S3InputFile.java
##########
@@ -20,12 +20,16 @@
 package org.apache.iceberg.aws.s3;
 
 import org.apache.iceberg.aws.AwsProperties;
+import org.apache.iceberg.encryption.NativeFileCryptoParameters;
+import org.apache.iceberg.encryption.NativelyEncryptedFile;
 import org.apache.iceberg.io.InputFile;
 import org.apache.iceberg.io.SeekableInputStream;
 import org.apache.iceberg.metrics.MetricsContext;
 import software.amazon.awssdk.services.s3.S3Client;
 
-public class S3InputFile extends BaseS3File implements InputFile {
+public class S3InputFile extends BaseS3File implements InputFile, NativelyEncryptedFile {
+  private NativeFileCryptoParameters nativeDecryptionParameters;

Review comment:
       here, we can use a more focused naming. `NativeFileCryptoParameters` are per-file, and are symmetric (applicable to both encryption and decryption). In an `InputFile` object, we can drop the 'file' part, and narrow this down to decryption-only. So the `nativeDecryptionParameters` name might be more streamlined and informative than a `nativeFileCryptoParameters`.
   But if Iceberg has a convention of matching the filed names to the class names, then sure, I'll make 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] jackye1995 edited a comment on pull request #2638: Core: Native encryption params

Posted by GitBox <gi...@apache.org>.
jackye1995 edited a comment on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-849001743


   The biggest question I have regarding this is that: should we have this concept of "native" encryption? 
   
   My original thinking is that we should use native encryption as long as the file format supports it and there are enough information provided in key_metadata.
   
   For example, a user can specify all those native configurations and enable it, but use Avro that does not have column encryption, then it still could not be applied.
   
   The only use case for not using native I think is for backwards compatibility with some other dependent systems. In that case, we only need a boolean config flag for the reader and writer, and I feel it is unnecessary to dedicate specific parameters only for native 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.

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] shangxinli commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r831154574



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;

Review comment:
       Given this is public method, should we do checkNotNull(keyBytes)?




-- 
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] shangxinli commented on a change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
shangxinli commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r831160912



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);

Review comment:
       Wonder if you want to define an Iceberg Crypto Runtime Exception just like you did for Parquet (ParquetCryptoRuntimeException). The reason is  RuntimeException is a very generic type and we will lose some meaning when converting from GeneralSecurityException. 

##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    public byte[] encrypt(byte[] plainText, byte[] aad) {
+      byte[] nonce = new byte[NONCE_LENGTH];

Review comment:
       In parquet, you have a check of excessive use of one single key(GCM_RANDOM_IV_SAME_KEY_MAX_OPS). Do you still want to do that here? 

##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);
+      }
+
+      // Add the nonce
+      System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH);
+
+      return cipherText;
+    }
+  }
+
+  public static class AesGcmDecryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+
+    public AesGcmDecryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+    }
+
+    public byte[] decrypt(byte[] ciphertext, byte[] aad)  {
+      int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH;
+      Preconditions.checkState(plainTextLength >= 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);
+
+      byte[] plainText = new byte[plainTextLength];
+      int inputLength = ciphertext.length - 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);
+      }  catch (AEADBadTagException e) {
+        throw new RuntimeException("GCM tag check failed", e);

Review comment:
       Can we add more information in this exception for debugging friendly? We had several issues earlier with this exception and felt hard to know what is going on. 

##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileKey;
+  private EncryptionAlgorithm fileEncryptionAlgorithm;
+
+  private NativeFileCryptoParameters(ByteBuffer fileKey, EncryptionAlgorithm fileEncryptionAlgorithm) {
+    Preconditions.checkState(fileKey != null, "File encryption key is not supplied");

Review comment:
       key length checking also?

##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    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;
+      byte[] cipherText = new byte[cipherTextLength];
+
+      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);
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to encrypt", e);

Review comment:
       Same comments above and other places.

##########
File path: core/src/main/java/org/apache/iceberg/encryption/EncryptionAlgorithm.java
##########
@@ -0,0 +1,52 @@
+/*
+ * 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;
+
+/**
+ * Algorithm supported for file encryption.
+ */
+public enum EncryptionAlgorithm {
+    /**
+     * Counter mode (CTR) allows fast encryption with high throughput.
+     * It is an encryption only cipher and does not ensure content integrity.
+     * Inputs to CTR cipher are:
+     *  1. encryption key
+     *  2. a 16-byte initialization vector (12-byte nonce, 4-byte counter)
+     *  3. plaintext data
+     */
+    AES_CTR,
+    /**
+     * Galois/Counter mode (GCM) combines CTR with the new Galois mode of authentication.
+     * It not only ensures data confidentiality, but also ensures data integrity.
+     * Inputs to GCM cipher are:
+     *  1. encryption key
+     *  2. a 12-byte initialization vector
+     *  3. additional authenticated data
+     *  4. plaintext data
+     */
+    AES_GCM,
+    /**
+     * A combination of GCM and CTR that can be used for file types like Parquet,
+     * so that all modules except pages are encrypted by GCM to ensure integrity,
+     * and CTR is used for efficient encryption of bulk data.
+     * The tradeoff is that attackers would be able to tamper page data.

Review comment:
       This comment is a little misleading if we don't say "attackers would be able to tamper data" for AES_CTR




-- 
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] jackye1995 commented on pull request #2638: Core: Native encryption params

Posted by GitBox <gi...@apache.org>.
jackye1995 commented on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-849001743


   The biggest question I have regarding this is that: should we have this concept of "native" encryption? 
   
   My original thinking is that we should use native encryption as long as the file format supports it and there are enough information provided in key_metadata.
   
   For example, a user can specify all those native configurations and enable it, but use Avro that does not have column encryption, then it still could not be applied.
   


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

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 #2638: Core: Native encryption params

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


   I've dug in the code a bit, there seems to be a practical solution to this. The workers (data/delete writers) can generate random DEKs  for each file - like we do today in PME - and pack them in the `ContentFile` objects for the delivery to the driver (manifest writer). The manifest writer will interact with a KMS to wrap the DEKs (per the single/double wrap design), and will store the result in the `key_material` field. I'll change the PR accordingly.
   As for the "native"-vs-general encryption - there are some interesting trade-offs. Both subjects TBD.


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

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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r788082408



##########
File path: api/src/main/java/org/apache/iceberg/io/InputFile.java
##########
@@ -59,4 +60,18 @@
    * @return true if the file exists, false otherwise
    */
   boolean exists();
+
+  // TODO remove this comment after review

Review comment:
       Brainstorming an idea with trait-like solution:
   1. Create a new interface for encryption file or reuse the interface `EncryptedInputFile`, which will have method `getNativeDecryptionParameters` and `setNativeDecryptionParameters`. 
   2. The subclass(HadoopInputFile, S3InputFile, etc) can implement both `InputFile` and the `EncryptedInputFile`.
   3. In case of the caller side, we can invoke it like this  `((EncryptedInputFile)inputFile).getNativeDecryptionParameters()`




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r795484016



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativelyEncryptedFile.java
##########
@@ -0,0 +1,29 @@
+/*
+ * 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;
+
+/**
+ * a minimum client interface to connect to a key management service (KMS).
+ */
+public interface NativelyEncryptedFile {

Review comment:
       NativeFileCryptoParameters is the only class planned for these properties; having an interface just for one implementation won't be efficient. Also, this is an internal class, not designed to be a part of Iceberg API.
   However, thinking about the KmsClient interface in the next PR - this one will have many implementations, and it is a part of the user-facing API in the envelope encryption. I'll move it to the API module. 




-- 
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 pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
flyrain commented on pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#issuecomment-1027485384


   Hi @ggershinsky, the PR LGTM. Can you check the CI failure?


-- 
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 #2638: Core: Encryption basics

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


   


-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r831944163



##########
File path: core/src/main/java/org/apache/iceberg/encryption/Ciphers.java
##########
@@ -0,0 +1,125 @@
+/*
+ * 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.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;
+
+public class Ciphers {
+  private static final int NONCE_LENGTH = 12;
+  private static final int GCM_TAG_LENGTH = 16;
+  private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH;
+
+  public static class AesGcmEncryptor {
+    private final SecretKeySpec aesKey;
+    private final Cipher cipher;
+    private final SecureRandom randomGenerator;
+
+    public AesGcmEncryptor(byte[] keyBytes) {
+      int keyLength = keyBytes.length;
+      Preconditions.checkArgument((keyLength == 16 || keyLength == 24 || keyLength == 32),
+          "Cannot use a key of length " + keyLength + " because AES only allows 16, 24 or 32 bytes");
+      this.aesKey = new SecretKeySpec(keyBytes, "AES");
+
+      try {
+        this.cipher = Cipher.getInstance("AES/GCM/NoPadding");
+      } catch (GeneralSecurityException e) {
+        throw new RuntimeException("Failed to create GCM cipher", e);
+      }
+
+      this.randomGenerator = new SecureRandom();
+    }
+
+    public byte[] encrypt(byte[] plainText, byte[] aad) {
+      byte[] nonce = new byte[NONCE_LENGTH];

Review comment:
       This Iceberg class is designed for encryption of keys, not for encryption of data; so the risk of exceeding 2 billion operations during a process lifetime (meaning creation of 2 billion parquet files), is not real.




-- 
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 change in pull request #2638: Core: Encryption basics

Posted by GitBox <gi...@apache.org>.
ggershinsky commented on a change in pull request #2638:
URL: https://github.com/apache/iceberg/pull/2638#discussion_r831949036



##########
File path: core/src/main/java/org/apache/iceberg/encryption/NativeFileCryptoParameters.java
##########
@@ -0,0 +1,74 @@
+/*
+ * 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.nio.ByteBuffer;
+import org.apache.iceberg.relocated.com.google.common.base.Preconditions;
+
+/**
+ * Barebone encryption parameters, one object per content file.
+ * Carries the file encryption key (later, will be extended with column keys and AAD prefix).
+ * Applicable only to formats with native encryption support (Parquet and ORC).
+ */
+public class NativeFileCryptoParameters {
+  private ByteBuffer fileKey;
+  private EncryptionAlgorithm fileEncryptionAlgorithm;
+
+  private NativeFileCryptoParameters(ByteBuffer fileKey, EncryptionAlgorithm fileEncryptionAlgorithm) {
+    Preconditions.checkState(fileKey != null, "File encryption key is not supplied");

Review comment:
       key length is checked elsewhere, we shouldn't run the checks in each method that passes a key.




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