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: [email protected] For queries about this service, please contact Infrastructure at: [email protected] --------------------------------------------------------------------- To unsubscribe, e-mail: [email protected] For additional commands, e-mail: [email protected]
