From 8a0c7a715b64bce3bdf6ab4087edc85b909e63da Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Wed, 6 Oct 2021 15:19:12 +0300 Subject: [PATCH 01/16] initial commit --- .../iceberg/encryption/AesGcmInputStream.java | 201 ++++++++++++++++++ .../encryption/AesGcmOutputStream.java | 150 +++++++++++++ 2 files changed, 351 insertions(+) create mode 100644 core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java create mode 100644 core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java new file mode 100644 index 000000000000..5ad0ac2cf614 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -0,0 +1,201 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.encryption; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.security.GeneralSecurityException; +import javax.crypto.Cipher; +import javax.crypto.SecretKey; +import javax.crypto.spec.GCMParameterSpec; +import javax.crypto.spec.SecretKeySpec; +import org.apache.iceberg.io.SeekableInputStream; + +import static org.apache.iceberg.encryption.AesGcmOutputStream.HEADER_SIZE_LENGTH; + +public class AesGcmInputStream extends SeekableInputStream { + private SeekableInputStream sourceStream; + private long netSourceFileSize; + + private Cipher gcmCipher; + private SecretKey key; + private byte[] nonce; + + private byte[] ciphertextBlockBuffer; + private int cipherBlockSize; + private int plainBlockSize; + private long plainStreamPosition; + private int currentBlockIndex; + private int currentOffsetInPlainBlock; + private int numberOfBlocks; + private int lastBlockSize; + private long plainStreamSize; + private byte[] fileAadPrefix; + + AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength, + byte[] aesKey, byte[] fileAadPrefix) throws IOException { + this.netSourceFileSize = sourceLength - HEADER_SIZE_LENGTH; + this.sourceStream = sourceStream; + byte[] blockSizeBytes = new byte[HEADER_SIZE_LENGTH]; + int fetched = sourceStream.read(blockSizeBytes); // TODO check + this.plainStreamPosition = 0; + this.fileAadPrefix = fileAadPrefix; + + plainBlockSize = ByteBuffer.wrap(blockSizeBytes).order(ByteOrder.LITTLE_ENDIAN).getInt(); + cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH; + + try { + gcmCipher = Cipher.getInstance("AES/GCM/NoPadding"); + } catch (GeneralSecurityException e) { + throw new IOException(e); + } + this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH]; + this.key = new SecretKeySpec(aesKey, "AES"); + this.ciphertextBlockBuffer = new byte[cipherBlockSize]; + this.currentBlockIndex = 0; + this.currentOffsetInPlainBlock = 0; + + numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize); + lastBlockSize = (int) (netSourceFileSize % cipherBlockSize); + if (lastBlockSize == 0) { + lastBlockSize = cipherBlockSize; + } else { + numberOfBlocks += 1; + } + + plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize + + (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH); + } + + public long plaintextStreamSize() { + return plainStreamSize; + } + + @Override + public int available() throws IOException { + return (int) (plainStreamSize - plainStreamPosition); + } + + @Override + public int read(byte[] b) throws IOException { + return read(b, 0, b.length); + } + + @Override + public int read(byte[] b, int off, int len) throws IOException { + if (len <= 0) { + throw new IOException("Negative read length " + len); + } + + if (available() <= 0) { + return -1; + } + + boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks; + int resultBufferOffset = off; + int remaining = len; + + while (remaining > 0) { + sourceStream.seek(HEADER_SIZE_LENGTH + currentBlockIndex * cipherBlockSize); + + int toLoad = lastBlock ? lastBlockSize : cipherBlockSize; + int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad); + if (loaded != toLoad) { + throw new IOException("Read " + loaded + " instead of " + toLoad); + } + + // Copy nonce + System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH); + + byte[] plaintextBlock = null; + try { + GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce); + gcmCipher.init(Cipher.DECRYPT_MODE, key, spec); + + // TODO byte[] aaD = calculateAAD(fileAadPrefix, currentBlockIndex); + + plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH, + toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH); + } catch (GeneralSecurityException e) { + throw new IOException("Failed to decrypt", e); + } + + int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock; + boolean finishTheBlock = remaining >= remainingInBlock; + int toCopy = finishTheBlock ? remainingInBlock : remaining; + + System.arraycopy(plaintextBlock, currentOffsetInPlainBlock, b, resultBufferOffset, toCopy); + remaining -= toCopy; + resultBufferOffset += toCopy; + currentOffsetInPlainBlock += toCopy; + boolean endOfStream = lastBlock && finishTheBlock; + if (endOfStream) { + break; + } + if (finishTheBlock) { + currentBlockIndex++; + currentOffsetInPlainBlock = 0; + lastBlock = currentBlockIndex + 1 == numberOfBlocks; + } + } + + plainStreamPosition += len - remaining; + return len - remaining; + } + + @Override + public void seek(long newPos) throws IOException { + currentBlockIndex = (int) (newPos / plainBlockSize); + currentOffsetInPlainBlock = (int) (newPos % plainBlockSize); + plainStreamPosition = newPos; + } + + @Override + public long getPos() throws IOException { + return plainStreamPosition; + } + + @Override + public int read() throws IOException { + throw new IOException("should not be called"); + } + + @Override + public void close() throws IOException { + sourceStream.close(); + } + + @Override + public synchronized void mark(int readlimit) { + // TODO implement + } + + @Override + public synchronized void reset() throws IOException { + // TODO implement + } + + @Override + public boolean markSupported() { + // TODO implement + return false; + } +} diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java new file mode 100644 index 000000000000..25db09948897 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -0,0 +1,150 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.encryption; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import java.security.GeneralSecurityException; +import java.security.InvalidAlgorithmParameterException; +import java.security.InvalidKeyException; +import java.security.SecureRandom; +import javax.crypto.Cipher; +import javax.crypto.SecretKey; +import javax.crypto.spec.GCMParameterSpec; +import javax.crypto.spec.SecretKeySpec; +import org.apache.iceberg.io.PositionOutputStream; + +public class AesGcmOutputStream extends PositionOutputStream { + // AES-GCM parameters + public static final int GCM_NONCE_LENGTH = 12; // in bytes + public static final int GCM_TAG_LENGTH = 16; // in bytes + public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH; + + static final int HEADER_SIZE_LENGTH = 4; + + private PositionOutputStream targetStream; + + private Cipher gcmCipher; + private SecureRandom random; + private SecretKey key; + private byte[] nonce; + + private int blockSize = 1024 * 1024; // TODO Make configurable + private byte[] plaintextBlockBuffer; + private int positionInBuffer; + private long streamPosition; + private int currentBlockIndex; + private byte[] fileAadPrefix; + + AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) throws IOException { + this.targetStream = targetStream; + + try { + gcmCipher = Cipher.getInstance("AES/GCM/NoPadding"); + } catch (GeneralSecurityException e) { + throw new IOException(e); + } + this.random = new SecureRandom(); + this.nonce = new byte[GCM_NONCE_LENGTH]; + this.key = new SecretKeySpec(aesKey, "AES"); + this.plaintextBlockBuffer = new byte[blockSize]; + this.positionInBuffer = 0; + this.streamPosition = 0; + this.currentBlockIndex = 0; + this.fileAadPrefix = fileAadPrefix; + + byte[] blockSizeBytes = ByteBuffer.allocate(HEADER_SIZE_LENGTH).order(ByteOrder.LITTLE_ENDIAN).putInt(blockSize).array(); + targetStream.write(blockSizeBytes); + } + + @Override + public void write(int b) throws IOException { + throw new IOException("should not be called"); + } + + @Override + public void write(byte[] b) throws IOException { + write(b, 0, b.length); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + int remaining = len; + int offset = off; + + while (remaining > 0) { + int freeBlockBytes = blockSize - positionInBuffer; + int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining; + + System.arraycopy(b, offset, plaintextBlockBuffer, positionInBuffer, toWrite); + positionInBuffer += toWrite; + if (positionInBuffer == blockSize) { + encryptAndWriteBlock(); + positionInBuffer = 0; + } + offset += toWrite; + remaining -= toWrite; + } + + streamPosition += len; + } + + @Override + public long getPos() throws IOException { + return streamPosition; + } + + @Override + public void flush() throws IOException { + targetStream.flush(); + } + + @Override + public void close() throws IOException { + if (positionInBuffer > 0) { + encryptAndWriteBlock(); + } + targetStream.close(); + } + + private void encryptAndWriteBlock() throws IOException { + random.nextBytes(nonce); + GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce); + // TODO byte[] aaD = calculateAAD(fileAadPrefix, currentBlockIndex); + try { + gcmCipher.init(Cipher.ENCRYPT_MODE, key, spec); + } catch (InvalidKeyException | InvalidAlgorithmParameterException e) { + throw new IOException("Failed to init GCM cipher", e); + } + + byte[] cipherText; + try { + cipherText = gcmCipher.doFinal(plaintextBlockBuffer, 0, positionInBuffer); + } catch (GeneralSecurityException e) { + throw new IOException("Failed to encrypt", e); + } + + currentBlockIndex++; + + targetStream.write(nonce); + targetStream.write(cipherText); + } +} From c4a9378b9dd3c5eb02fd1a4744e57e6915c62e29 Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Tue, 21 Jun 2022 10:19:20 +0300 Subject: [PATCH 02/16] update --- .../iceberg/encryption/AesGcmInputFile.java | 67 +++++++++++ .../iceberg/encryption/AesGcmInputStream.java | 47 +++++--- .../iceberg/encryption/AesGcmOutputFile.java | 64 +++++++++++ .../encryption/AesGcmOutputStream.java | 43 ++++++-- .../iceberg/encryption/TestGcmStreams.java | 104 ++++++++++++++++++ 5 files changed, 297 insertions(+), 28 deletions(-) create mode 100644 core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java create mode 100644 core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java create mode 100644 core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java new file mode 100644 index 000000000000..e27c8c7faeef --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java @@ -0,0 +1,67 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.encryption; + +import java.io.IOException; +import java.io.UncheckedIOException; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class AesGcmInputFile implements InputFile { + private InputFile sourceFile; + private byte[] dataKey; + private long plaintextLength; + + public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) { + this.sourceFile = sourceFile; + this.dataKey = dataKey; + this.plaintextLength = -1; + } + + @Override + public long getLength() { + Preconditions.checkArgument(plaintextLength >= 0, "Length is known after new stream is created"); + + return plaintextLength; + } + + @Override + public SeekableInputStream newStream() { + AesGcmInputStream result; + try { + result = new AesGcmInputStream(sourceFile.newStream(), sourceFile.getLength(), dataKey, null); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + plaintextLength = result.plaintextStreamSize(); + return result; + } + + @Override + public String location() { + return sourceFile.location(); + } + + @Override + public boolean exists() { + return sourceFile.exists(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index 5ad0ac2cf614..9fd74db15cbe 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -23,13 +23,13 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.security.GeneralSecurityException; +import java.util.Arrays; import javax.crypto.Cipher; import javax.crypto.SecretKey; import javax.crypto.spec.GCMParameterSpec; import javax.crypto.spec.SecretKeySpec; import org.apache.iceberg.io.SeekableInputStream; - -import static org.apache.iceberg.encryption.AesGcmOutputStream.HEADER_SIZE_LENGTH; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class AesGcmInputStream extends SeekableInputStream { private SeekableInputStream sourceStream; @@ -52,14 +52,23 @@ public class AesGcmInputStream extends SeekableInputStream { AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAadPrefix) throws IOException { - this.netSourceFileSize = sourceLength - HEADER_SIZE_LENGTH; + this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH; this.sourceStream = sourceStream; - byte[] blockSizeBytes = new byte[HEADER_SIZE_LENGTH]; - int fetched = sourceStream.read(blockSizeBytes); // TODO check + byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH]; + int fetched = sourceStream.read(prefixBytes); + Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH, + "Insufficient read " + fetched); this.plainStreamPosition = 0; this.fileAadPrefix = fileAadPrefix; - plainBlockSize = ByteBuffer.wrap(blockSizeBytes).order(ByteOrder.LITTLE_ENDIAN).getInt(); + byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length]; + System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length); + + Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic), + "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING); + + plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4) + .order(ByteOrder.LITTLE_ENDIAN).getInt(); cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH; try { @@ -91,7 +100,7 @@ public long plaintextStreamSize() { @Override public int available() throws IOException { - return (int) (plainStreamSize - plainStreamPosition); + return Math.toIntExact(plainStreamSize - plainStreamPosition); } @Override @@ -113,9 +122,9 @@ public int read(byte[] b, int off, int len) throws IOException { int resultBufferOffset = off; int remaining = len; - while (remaining > 0) { - sourceStream.seek(HEADER_SIZE_LENGTH + currentBlockIndex * cipherBlockSize); + sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize); + while (remaining > 0) { int toLoad = lastBlock ? lastBlockSize : cipherBlockSize; int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad); if (loaded != toLoad) { @@ -125,12 +134,12 @@ public int read(byte[] b, int off, int len) throws IOException { // Copy nonce System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH); - byte[] plaintextBlock = null; + byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex); + byte[] plaintextBlock; try { GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce); gcmCipher.init(Cipher.DECRYPT_MODE, key, spec); - - // TODO byte[] aaD = calculateAAD(fileAadPrefix, currentBlockIndex); + gcmCipher.updateAAD(aad); plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH, toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH); @@ -163,8 +172,11 @@ public int read(byte[] b, int off, int len) throws IOException { @Override public void seek(long newPos) throws IOException { - currentBlockIndex = (int) (newPos / plainBlockSize); - currentOffsetInPlainBlock = (int) (newPos % plainBlockSize); + if (newPos >= plainStreamSize) { + throw new IOException("At or beyond max stream size " + plainStreamSize + ", " + newPos); + } + currentBlockIndex = Math.toIntExact(newPos / plainBlockSize); + currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize); plainStreamPosition = newPos; } @@ -175,7 +187,7 @@ public long getPos() throws IOException { @Override public int read() throws IOException { - throw new IOException("should not be called"); + throw new UnsupportedOperationException(); } @Override @@ -185,17 +197,16 @@ public void close() throws IOException { @Override public synchronized void mark(int readlimit) { - // TODO implement + throw new UnsupportedOperationException(); } @Override public synchronized void reset() throws IOException { - // TODO implement + throw new UnsupportedOperationException(); } @Override public boolean markSupported() { - // TODO implement return false; } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java new file mode 100644 index 000000000000..670079f628cd --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java @@ -0,0 +1,64 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.encryption; + +import java.io.IOException; +import java.io.UncheckedIOException; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.PositionOutputStream; + +public class AesGcmOutputFile implements OutputFile { + private OutputFile targetFile; + private byte[] dataKey; + + public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey) { + this.targetFile = targetFile; + this.dataKey = dataKey; + } + + @Override + public PositionOutputStream create() { + try { + return new AesGcmOutputStream(targetFile.create(), dataKey, null); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public PositionOutputStream createOrOverwrite() { + try { + return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, null); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } + + @Override + public String location() { + return targetFile.location(); + } + + @Override + public InputFile toInputFile() { + throw new UnsupportedOperationException(); + } +} diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index 25db09948897..37bdd8dd9552 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -22,6 +22,7 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; +import java.nio.charset.StandardCharsets; import java.security.GeneralSecurityException; import java.security.InvalidAlgorithmParameterException; import java.security.InvalidKeyException; @@ -31,14 +32,18 @@ import javax.crypto.spec.GCMParameterSpec; import javax.crypto.spec.SecretKeySpec; import org.apache.iceberg.io.PositionOutputStream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.primitives.Ints; public class AesGcmOutputStream extends PositionOutputStream { // AES-GCM parameters public static final int GCM_NONCE_LENGTH = 12; // in bytes public static final int GCM_TAG_LENGTH = 16; // in bytes public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH; + public static final String MAGIC_STRING = "GCM1"; - static final int HEADER_SIZE_LENGTH = 4; + static final byte[] MAGIC_ARRAY = MAGIC_STRING.getBytes(StandardCharsets.UTF_8); + static final int PREFIX_LENGTH = MAGIC_ARRAY.length + 4; // magic_len + block_size_len private PositionOutputStream targetStream; @@ -47,7 +52,7 @@ public class AesGcmOutputStream extends PositionOutputStream { private SecretKey key; private byte[] nonce; - private int blockSize = 1024 * 1024; // TODO Make configurable + private int blockSize = 1024 * 1024; private byte[] plaintextBlockBuffer; private int positionInBuffer; private long streamPosition; @@ -56,7 +61,6 @@ public class AesGcmOutputStream extends PositionOutputStream { AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) throws IOException { this.targetStream = targetStream; - try { gcmCipher = Cipher.getInstance("AES/GCM/NoPadding"); } catch (GeneralSecurityException e) { @@ -71,13 +75,16 @@ public class AesGcmOutputStream extends PositionOutputStream { this.currentBlockIndex = 0; this.fileAadPrefix = fileAadPrefix; - byte[] blockSizeBytes = ByteBuffer.allocate(HEADER_SIZE_LENGTH).order(ByteOrder.LITTLE_ENDIAN).putInt(blockSize).array(); - targetStream.write(blockSizeBytes); + byte[] prefixBytes = ByteBuffer.allocate(PREFIX_LENGTH).order(ByteOrder.LITTLE_ENDIAN) + .put(MAGIC_ARRAY) + .putInt(blockSize) + .array(); + targetStream.write(prefixBytes); } @Override public void write(int b) throws IOException { - throw new IOException("should not be called"); + throw new UnsupportedOperationException(); } @Override @@ -128,23 +135,39 @@ public void close() throws IOException { private void encryptAndWriteBlock() throws IOException { random.nextBytes(nonce); GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce); - // TODO byte[] aaD = calculateAAD(fileAadPrefix, currentBlockIndex); try { gcmCipher.init(Cipher.ENCRYPT_MODE, key, spec); } catch (InvalidKeyException | InvalidAlgorithmParameterException e) { throw new IOException("Failed to init GCM cipher", e); } - byte[] cipherText; + byte[] aad = calculateAAD(fileAadPrefix, currentBlockIndex); + gcmCipher.updateAAD(aad); + + byte[] cipherText = new byte[GCM_NONCE_LENGTH + positionInBuffer + GCM_TAG_LENGTH]; + System.arraycopy(nonce, 0, cipherText, 0, GCM_NONCE_LENGTH); try { - cipherText = gcmCipher.doFinal(plaintextBlockBuffer, 0, positionInBuffer); + int encrypted = gcmCipher.doFinal(plaintextBlockBuffer, 0, positionInBuffer, cipherText, GCM_NONCE_LENGTH); + Preconditions.checkArgument((encrypted == (positionInBuffer + GCM_TAG_LENGTH)), + "Wrong length of encrypted output: " + encrypted + " vs " + (positionInBuffer + GCM_TAG_LENGTH)); } catch (GeneralSecurityException e) { throw new IOException("Failed to encrypt", e); } currentBlockIndex++; - targetStream.write(nonce); targetStream.write(cipherText); } + + static byte[] calculateAAD(byte[] fileAadPrefix, int currentBlockIndex) { + byte[] blockAAD = Ints.toByteArray(currentBlockIndex); + if (null == fileAadPrefix) { + return blockAAD; + } else { + byte[] aad = new byte[fileAadPrefix.length + 4]; + System.arraycopy(fileAadPrefix, 0, aad, 0, fileAadPrefix.length); + System.arraycopy(blockAAD, 0, aad, fileAadPrefix.length, 4); + return aad; + } + } } diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java new file mode 100644 index 000000000000..0b5a66a1e2ba --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -0,0 +1,104 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ + +package org.apache.iceberg.encryption; + + +import java.io.File; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.util.Random; +import org.apache.iceberg.Files; +import org.apache.iceberg.io.PositionOutputStream; +import org.apache.iceberg.io.SeekableInputStream; +import org.junit.Assert; +import org.junit.Rule; +import org.junit.Test; +import org.junit.rules.TemporaryFolder; + +public class TestGcmStreams { + + @Rule + public TemporaryFolder temp = new TemporaryFolder(); + + @Test + public void testRandomWriteRead() throws IOException { + Random random = new Random(); + int testFileSize = 1000000; + byte[] testFileContents = new byte[testFileSize]; + random.nextBytes(testFileContents); + int[] aesKeyLengthArray = {16, 24, 32}; + for (int keyLength : aesKeyLengthArray) { + byte[] key = new byte[keyLength]; + random.nextBytes(key); + File testFile = temp.newFile(); + + AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key); + PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); + + int maxChunkLen = testFileSize / 5; + int offset = 0; + int left = testFileSize; + + while (left > 0) { + int chunkLen = random.nextInt(maxChunkLen); + if (chunkLen > left) { + chunkLen = left; + } + encryptedStream.write(testFileContents, offset, chunkLen); + offset += chunkLen; + Assert.assertEquals("Position", offset, encryptedStream.getPos()); + left -= chunkLen; + + if (left == 0) { + break; + } + } + encryptedStream.close(); + + AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key); + SeekableInputStream decryptedStream = decryptedFile.newStream(); + + Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); + + byte[] chunk = new byte[testFileSize]; + + for (int n = 0; n < 1000; n++) { + int chunkLen = random.nextInt(testFileSize); + int pos = random.nextInt(testFileSize); + left = testFileSize - pos; + if (left < chunkLen) { + chunkLen = left; + } + + decryptedStream.seek(pos); + int len = decryptedStream.read(chunk, 0, chunkLen); + + Assert.assertEquals("Read length", len, chunkLen); + + ByteBuffer bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); + ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, pos, chunkLen); + + Assert.assertEquals("Read contents", bb1, bb2); + } + + decryptedStream.close(); + } + } +} From 51a5b6e00c195f9eaa79b06d4dc18145d88be351 Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Thu, 23 Jun 2022 13:44:11 +0300 Subject: [PATCH 03/16] address review 2 --- .../iceberg/encryption/AesGcmInputFile.java | 14 +- .../iceberg/encryption/AesGcmInputStream.java | 174 ++++++++---------- .../iceberg/encryption/AesGcmOutputFile.java | 8 +- .../encryption/AesGcmOutputStream.java | 98 ++-------- .../apache/iceberg/encryption/Ciphers.java | 60 +++++- .../iceberg/encryption/TestGcmStreams.java | 126 ++++++++++--- 6 files changed, 258 insertions(+), 222 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java index e27c8c7faeef..25208da5e41f 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java @@ -23,11 +23,10 @@ import java.io.UncheckedIOException; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.SeekableInputStream; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class AesGcmInputFile implements InputFile { - private InputFile sourceFile; - private byte[] dataKey; + private final InputFile sourceFile; + private final byte[] dataKey; private long plaintextLength; public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) { @@ -38,8 +37,13 @@ public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) { @Override public long getLength() { - Preconditions.checkArgument(plaintextLength >= 0, "Length is known after new stream is created"); - + if (plaintextLength == -1) { + try { + this.newStream().close(); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + } return plaintextLength; } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index 9fd74db15cbe..b6f750c2c7a4 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -20,78 +20,81 @@ package org.apache.iceberg.encryption; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.security.GeneralSecurityException; import java.util.Arrays; -import javax.crypto.Cipher; -import javax.crypto.SecretKey; -import javax.crypto.spec.GCMParameterSpec; -import javax.crypto.spec.SecretKeySpec; import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class AesGcmInputStream extends SeekableInputStream { - private SeekableInputStream sourceStream; - private long netSourceFileSize; + private final SeekableInputStream sourceStream; + private final boolean emptyCipherStream; + private final long netSourceFileSize; + private final Ciphers.AesGcmDecryptor gcmDecryptor; + private final byte[] ciphertextBlockBuffer; + private final int cipherBlockSize; + private final int plainBlockSize; + private final int numberOfBlocks; + private final int lastCipherBlockSize; + private final long plainStreamSize; + private final byte[] fileAadPrefix; - private Cipher gcmCipher; - private SecretKey key; - private byte[] nonce; - - private byte[] ciphertextBlockBuffer; - private int cipherBlockSize; - private int plainBlockSize; private long plainStreamPosition; private int currentBlockIndex; private int currentOffsetInPlainBlock; - private int numberOfBlocks; - private int lastBlockSize; - private long plainStreamSize; - private byte[] fileAadPrefix; AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAadPrefix) throws IOException { - this.netSourceFileSize = sourceLength - AesGcmOutputStream.PREFIX_LENGTH; + this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH; + Preconditions.checkArgument(netSourceFileSize >= 0, + "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted"); + + this.emptyCipherStream = (0 == netSourceFileSize); this.sourceStream = sourceStream; - byte[] prefixBytes = new byte[AesGcmOutputStream.PREFIX_LENGTH]; + byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH]; int fetched = sourceStream.read(prefixBytes); - Preconditions.checkArgument(fetched == AesGcmOutputStream.PREFIX_LENGTH, - "Insufficient read " + fetched); - this.plainStreamPosition = 0; - this.fileAadPrefix = fileAadPrefix; - - byte[] magic = new byte[AesGcmOutputStream.MAGIC_ARRAY.length]; - System.arraycopy(prefixBytes, 0, magic, 0, AesGcmOutputStream.MAGIC_ARRAY.length); - - Preconditions.checkArgument(Arrays.equals(AesGcmOutputStream.MAGIC_ARRAY, magic), - "File with wrong magic string. Should start with " + AesGcmOutputStream.MAGIC_STRING); - - plainBlockSize = ByteBuffer.wrap(prefixBytes, AesGcmOutputStream.MAGIC_ARRAY.length, 4) - .order(ByteOrder.LITTLE_ENDIAN).getInt(); - cipherBlockSize = plainBlockSize + AesGcmOutputStream.GCM_NONCE_LENGTH + AesGcmOutputStream.GCM_TAG_LENGTH; - - try { - gcmCipher = Cipher.getInstance("AES/GCM/NoPadding"); - } catch (GeneralSecurityException e) { - throw new IOException(e); - } - this.nonce = new byte[AesGcmOutputStream.GCM_NONCE_LENGTH]; - this.key = new SecretKeySpec(aesKey, "AES"); - this.ciphertextBlockBuffer = new byte[cipherBlockSize]; - this.currentBlockIndex = 0; - this.currentOffsetInPlainBlock = 0; - - numberOfBlocks = (int) (netSourceFileSize / cipherBlockSize); - lastBlockSize = (int) (netSourceFileSize % cipherBlockSize); - if (lastBlockSize == 0) { - lastBlockSize = cipherBlockSize; + Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH, + "Insufficient read " + fetched + + ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH); + + byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length]; + System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length); + Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic), + "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING); + + if (!emptyCipherStream) { + this.plainStreamPosition = 0; + this.fileAadPrefix = fileAadPrefix; + gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey); + plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4) + .order(ByteOrder.LITTLE_ENDIAN).getInt(); + Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize); + + cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH; + this.ciphertextBlockBuffer = new byte[cipherBlockSize]; + this.currentBlockIndex = 0; + this.currentOffsetInPlainBlock = 0; + + int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize); + int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize); + boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); + numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1; + lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0 + int plainBytesInLastBlock = fullBlocksOnly ? 0 : + (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH); + plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock; } else { - numberOfBlocks += 1; + plainStreamSize = 0; + + gcmDecryptor = null; + ciphertextBlockBuffer = null; + cipherBlockSize = -1; + plainBlockSize = -1; + numberOfBlocks = -1; + lastCipherBlockSize = -1; + this.fileAadPrefix = null; } - - plainStreamSize = (numberOfBlocks - 1L) * plainBlockSize + - (lastBlockSize - AesGcmOutputStream.GCM_NONCE_LENGTH - AesGcmOutputStream.GCM_TAG_LENGTH); } public long plaintextStreamSize() { @@ -100,17 +103,18 @@ public long plaintextStreamSize() { @Override public int available() throws IOException { - return Math.toIntExact(plainStreamSize - plainStreamPosition); - } - - @Override - public int read(byte[] b) throws IOException { - return read(b, 0, b.length); + long maxAvailable = plainStreamSize - plainStreamPosition; + // See InputStream.available contract + if (maxAvailable >= Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } else { + return (int) maxAvailable; + } } @Override public int read(byte[] b, int off, int len) throws IOException { - if (len <= 0) { + if (len < 0) { throw new IOException("Negative read length " + len); } @@ -118,34 +122,21 @@ public int read(byte[] b, int off, int len) throws IOException { return -1; } - boolean lastBlock = currentBlockIndex + 1 == numberOfBlocks; + boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks); int resultBufferOffset = off; int remaining = len; - sourceStream.seek(AesGcmOutputStream.PREFIX_LENGTH + currentBlockIndex * cipherBlockSize); + sourceStream.seek(Ciphers.GCM_STREAM_PREFIX_LENGTH + currentBlockIndex * cipherBlockSize); while (remaining > 0) { - int toLoad = lastBlock ? lastBlockSize : cipherBlockSize; + int toLoad = lastBlock ? lastCipherBlockSize : cipherBlockSize; int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad); if (loaded != toLoad) { - throw new IOException("Read " + loaded + " instead of " + toLoad); + throw new IOException("Should read " + toLoad + " bytes, but got only " + loaded + " bytes"); } - // Copy nonce - System.arraycopy(ciphertextBlockBuffer, 0, nonce, 0, AesGcmOutputStream.GCM_NONCE_LENGTH); - - byte[] aad = AesGcmOutputStream.calculateAAD(fileAadPrefix, currentBlockIndex); - byte[] plaintextBlock; - try { - GCMParameterSpec spec = new GCMParameterSpec(AesGcmOutputStream.GCM_TAG_LENGTH_BITS, nonce); - gcmCipher.init(Cipher.DECRYPT_MODE, key, spec); - gcmCipher.updateAAD(aad); - - plaintextBlock = gcmCipher.doFinal(ciphertextBlockBuffer, AesGcmOutputStream.GCM_NONCE_LENGTH, - toLoad - AesGcmOutputStream.GCM_NONCE_LENGTH); - } catch (GeneralSecurityException e) { - throw new IOException("Failed to decrypt", e); - } + byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex); + byte[] plaintextBlock = gcmDecryptor.decrypt(ciphertextBlockBuffer, 0, toLoad, aad); int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock; boolean finishTheBlock = remaining >= remainingInBlock; @@ -162,7 +153,7 @@ public int read(byte[] b, int off, int len) throws IOException { if (finishTheBlock) { currentBlockIndex++; currentOffsetInPlainBlock = 0; - lastBlock = currentBlockIndex + 1 == numberOfBlocks; + lastBlock = (currentBlockIndex + 1 == numberOfBlocks); } } @@ -172,8 +163,10 @@ public int read(byte[] b, int off, int len) throws IOException { @Override public void seek(long newPos) throws IOException { - if (newPos >= plainStreamSize) { - throw new IOException("At or beyond max stream size " + plainStreamSize + ", " + newPos); + if (newPos < 0) { + throw new IOException("Negative new position " + newPos); + } else if (newPos > plainStreamSize) { + throw new IOException("New position " + newPos + " exceeds the max stream size " + plainStreamSize); } currentBlockIndex = Math.toIntExact(newPos / plainBlockSize); currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize); @@ -194,19 +187,4 @@ public int read() throws IOException { public void close() throws IOException { sourceStream.close(); } - - @Override - public synchronized void mark(int readlimit) { - throw new UnsupportedOperationException(); - } - - @Override - public synchronized void reset() throws IOException { - throw new UnsupportedOperationException(); - } - - @Override - public boolean markSupported() { - return false; - } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java index 670079f628cd..865d4a0eeae0 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java @@ -26,8 +26,8 @@ import org.apache.iceberg.io.PositionOutputStream; public class AesGcmOutputFile implements OutputFile { - private OutputFile targetFile; - private byte[] dataKey; + private final OutputFile targetFile; + private final byte[] dataKey; public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey) { this.targetFile = targetFile; @@ -39,7 +39,7 @@ public PositionOutputStream create() { try { return new AesGcmOutputStream(targetFile.create(), dataKey, null); } catch (IOException e) { - throw new UncheckedIOException(e); + throw new UncheckedIOException("Failed to create GCM stream for " + targetFile.location(), e); } } @@ -48,7 +48,7 @@ public PositionOutputStream createOrOverwrite() { try { return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, null); } catch (IOException e) { - throw new UncheckedIOException(e); + throw new UncheckedIOException("Failed to create or overwrite GCM stream for " + targetFile.location(), e); } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index 37bdd8dd9552..4b8c147af8fe 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -22,62 +22,32 @@ import java.io.IOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.nio.charset.StandardCharsets; -import java.security.GeneralSecurityException; -import java.security.InvalidAlgorithmParameterException; -import java.security.InvalidKeyException; -import java.security.SecureRandom; -import javax.crypto.Cipher; -import javax.crypto.SecretKey; -import javax.crypto.spec.GCMParameterSpec; -import javax.crypto.spec.SecretKeySpec; import org.apache.iceberg.io.PositionOutputStream; -import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.primitives.Ints; public class AesGcmOutputStream extends PositionOutputStream { - // AES-GCM parameters - public static final int GCM_NONCE_LENGTH = 12; // in bytes - public static final int GCM_TAG_LENGTH = 16; // in bytes - public static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH; - public static final String MAGIC_STRING = "GCM1"; + public static final int plainBlockSize = 1024 * 1024; - static final byte[] MAGIC_ARRAY = MAGIC_STRING.getBytes(StandardCharsets.UTF_8); - static final int PREFIX_LENGTH = MAGIC_ARRAY.length + 4; // magic_len + block_size_len + private final Ciphers.AesGcmEncryptor gcmEncryptor; + private final PositionOutputStream targetStream; + private final byte[] plaintextBlockBuffer; + private final byte[] fileAadPrefix; - private PositionOutputStream targetStream; - - private Cipher gcmCipher; - private SecureRandom random; - private SecretKey key; - private byte[] nonce; - - private int blockSize = 1024 * 1024; - private byte[] plaintextBlockBuffer; private int positionInBuffer; private long streamPosition; private int currentBlockIndex; - private byte[] fileAadPrefix; AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) throws IOException { this.targetStream = targetStream; - try { - gcmCipher = Cipher.getInstance("AES/GCM/NoPadding"); - } catch (GeneralSecurityException e) { - throw new IOException(e); - } - this.random = new SecureRandom(); - this.nonce = new byte[GCM_NONCE_LENGTH]; - this.key = new SecretKeySpec(aesKey, "AES"); - this.plaintextBlockBuffer = new byte[blockSize]; + this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey); + this.plaintextBlockBuffer = new byte[plainBlockSize]; this.positionInBuffer = 0; this.streamPosition = 0; this.currentBlockIndex = 0; this.fileAadPrefix = fileAadPrefix; - byte[] prefixBytes = ByteBuffer.allocate(PREFIX_LENGTH).order(ByteOrder.LITTLE_ENDIAN) - .put(MAGIC_ARRAY) - .putInt(blockSize) + byte[] prefixBytes = ByteBuffer.allocate(Ciphers.GCM_STREAM_PREFIX_LENGTH).order(ByteOrder.LITTLE_ENDIAN) + .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) + .putInt(plainBlockSize) .array(); targetStream.write(prefixBytes); } @@ -87,23 +57,21 @@ public void write(int b) throws IOException { throw new UnsupportedOperationException(); } - @Override - public void write(byte[] b) throws IOException { - write(b, 0, b.length); - } - @Override public void write(byte[] b, int off, int len) throws IOException { + if (b.length - off < len) { + throw new IOException("Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len); + } int remaining = len; int offset = off; while (remaining > 0) { - int freeBlockBytes = blockSize - positionInBuffer; + int freeBlockBytes = plainBlockSize - positionInBuffer; int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining; System.arraycopy(b, offset, plaintextBlockBuffer, positionInBuffer, toWrite); positionInBuffer += toWrite; - if (positionInBuffer == blockSize) { + if (positionInBuffer == plainBlockSize) { encryptAndWriteBlock(); positionInBuffer = 0; } @@ -133,41 +101,9 @@ public void close() throws IOException { } private void encryptAndWriteBlock() throws IOException { - random.nextBytes(nonce); - GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce); - try { - gcmCipher.init(Cipher.ENCRYPT_MODE, key, spec); - } catch (InvalidKeyException | InvalidAlgorithmParameterException e) { - throw new IOException("Failed to init GCM cipher", e); - } - - byte[] aad = calculateAAD(fileAadPrefix, currentBlockIndex); - gcmCipher.updateAAD(aad); - - byte[] cipherText = new byte[GCM_NONCE_LENGTH + positionInBuffer + GCM_TAG_LENGTH]; - System.arraycopy(nonce, 0, cipherText, 0, GCM_NONCE_LENGTH); - try { - int encrypted = gcmCipher.doFinal(plaintextBlockBuffer, 0, positionInBuffer, cipherText, GCM_NONCE_LENGTH); - Preconditions.checkArgument((encrypted == (positionInBuffer + GCM_TAG_LENGTH)), - "Wrong length of encrypted output: " + encrypted + " vs " + (positionInBuffer + GCM_TAG_LENGTH)); - } catch (GeneralSecurityException e) { - throw new IOException("Failed to encrypt", e); - } - + byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex); + byte[] cipherText = gcmEncryptor.encrypt(plaintextBlockBuffer, 0, positionInBuffer, aad); currentBlockIndex++; - targetStream.write(cipherText); } - - static byte[] calculateAAD(byte[] fileAadPrefix, int currentBlockIndex) { - byte[] blockAAD = Ints.toByteArray(currentBlockIndex); - if (null == fileAadPrefix) { - return blockAAD; - } else { - byte[] aad = new byte[fileAadPrefix.length + 4]; - System.arraycopy(fileAadPrefix, 0, aad, 0, fileAadPrefix.length); - System.arraycopy(blockAAD, 0, aad, fileAadPrefix.length, 4); - return aad; - } - } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index 6eecefda472c..6b5436cbf2cb 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.encryption; +import java.nio.charset.StandardCharsets; import java.security.GeneralSecurityException; import java.security.SecureRandom; import javax.crypto.AEADBadTagException; @@ -25,12 +26,21 @@ import javax.crypto.spec.GCMParameterSpec; import javax.crypto.spec.SecretKeySpec; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.primitives.Ints; public class Ciphers { - private static final int NONCE_LENGTH = 12; - private static final int GCM_TAG_LENGTH = 16; + public static final int NONCE_LENGTH = 12; + public static final int GCM_TAG_LENGTH = 16; + public static final String GCM_STREAM_MAGIC_STRING = "GCM1"; + + static final byte[] GCM_STREAM_MAGIC_ARRAY = GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8); + static final int GCM_STREAM_PREFIX_LENGTH = GCM_STREAM_MAGIC_ARRAY.length + 4; // magic_len + block_size_len + private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH; + private Ciphers() { + } + public static class AesGcmEncryptor { private final SecretKeySpec aesKey; private final Cipher cipher; @@ -55,10 +65,14 @@ public AesGcmEncryptor(byte[] keyBytes) { this.randomGenerator = new SecureRandom(); } - public byte[] encrypt(byte[] plainText, byte[] aad) { + public byte[] encrypt(byte[] plaintext, byte[] aad) { + return encrypt(plaintext, 0, plaintext.length, aad); + } + + public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) { byte[] nonce = new byte[NONCE_LENGTH]; randomGenerator.nextBytes(nonce); - int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH; + int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH; byte[] cipherText = new byte[cipherTextLength]; try { @@ -67,7 +81,7 @@ public byte[] encrypt(byte[] plainText, byte[] aad) { if (null != aad) { cipher.updateAAD(aad); } - cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH); + cipher.doFinal(plaintext, plaintextOffset, plaintextLength, cipherText, NONCE_LENGTH); } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to encrypt", e); } @@ -100,6 +114,7 @@ public AesGcmDecryptor(byte[] keyBytes) { } } +<<<<<<< HEAD public byte[] decrypt(byte[] ciphertext, byte[] aad) { int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH; Preconditions.checkState( @@ -108,30 +123,57 @@ public byte[] decrypt(byte[] ciphertext, byte[] aad) { + ciphertext.length + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" + " with AES GCM cipher"); +======= + public byte[] decrypt(byte[] ciphertext, byte[] aad) { + return decrypt(ciphertext, 0, ciphertext.length, aad); + } + + public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) { + Preconditions.checkState(ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1, + "Cannot decrypt cipher text of length " + ciphertext.length + + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" + + " with AES GCM cipher"); +>>>>>>> 52b944a45 (address review 2) // Get the nonce from ciphertext byte[] nonce = new byte[NONCE_LENGTH]; - System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH); + System.arraycopy(ciphertext, ciphertextOffset, nonce, 0, NONCE_LENGTH); - byte[] plainText = new byte[plainTextLength]; - int inputLength = ciphertext.length - NONCE_LENGTH; + int inputLength = ciphertextLength - NONCE_LENGTH; try { GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce); cipher.init(Cipher.DECRYPT_MODE, aesKey, spec); if (null != aad) { cipher.updateAAD(aad); } +<<<<<<< HEAD cipher.doFinal(ciphertext, NONCE_LENGTH, inputLength, plainText, 0); } catch (AEADBadTagException e) { throw new RuntimeException( "GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" + "data. AES GCM doesn't differentiate between these two.. ", e); +======= + return cipher.doFinal(ciphertext, ciphertextOffset + NONCE_LENGTH, inputLength); + } catch (AEADBadTagException e) { + throw new RuntimeException("GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" + + "data. AES GCM doesn't differentiate between these two.", e); +>>>>>>> 52b944a45 (address review 2) } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to decrypt", e); } + } + } - return plainText; + static byte[] streamBlockAAD(byte[] fileAadPrefix, int currentBlockIndex) { + byte[] blockAAD = Ints.toByteArray(currentBlockIndex); + if (null == fileAadPrefix) { + return blockAAD; + } else { + byte[] aad = new byte[fileAadPrefix.length + 4]; + System.arraycopy(fileAadPrefix, 0, aad, 0, fileAadPrefix.length); + System.arraycopy(blockAAD, 0, aad, fileAadPrefix.length, 4); + return aad; } } } diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java index 0b5a66a1e2ba..76171c60c33e 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -19,7 +19,6 @@ package org.apache.iceberg.encryption; - import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -37,27 +36,107 @@ public class TestGcmStreams { @Rule public TemporaryFolder temp = new TemporaryFolder(); + @Test + public void testEmptyFile() throws IOException { + Random random = new Random(); + byte[] key = new byte[16]; + random.nextBytes(key); + File testFile = temp.newFile(); + AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key); + PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); + encryptedStream.close(); + + AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key); + SeekableInputStream decryptedStream = decryptedFile.newStream(); + + Assert.assertEquals("File size", 0, decryptedFile.getLength()); + decryptedStream.close(); + } + @Test public void testRandomWriteRead() throws IOException { Random random = new Random(); - int testFileSize = 1000000; - byte[] testFileContents = new byte[testFileSize]; - random.nextBytes(testFileContents); - int[] aesKeyLengthArray = {16, 24, 32}; - for (int keyLength : aesKeyLengthArray) { - byte[] key = new byte[keyLength]; + int smallerThanBlock = (int) (AesGcmOutputStream.plainBlockSize * 0.5); + int largerThanBlock = (int) (AesGcmOutputStream.plainBlockSize * 1.5); + int alignedWithBlock = AesGcmOutputStream.plainBlockSize; + int[] testFileSizes = {smallerThanBlock, largerThanBlock, alignedWithBlock, alignedWithBlock - 1, + alignedWithBlock + 1}; + for (int testFileSize : testFileSizes) { + byte[] testFileContents = new byte[testFileSize]; + random.nextBytes(testFileContents); + int[] aesKeyLengthArray = {16, 24, 32}; + for (int keyLength : aesKeyLengthArray) { + byte[] key = new byte[keyLength]; + random.nextBytes(key); + File testFile = temp.newFile(); + + AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key); + PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); + + int maxChunkLen = testFileSize / 5; + int offset = 0; + int left = testFileSize; + + while (left > 0) { + int chunkLen = random.nextInt(maxChunkLen); + if (chunkLen > left) { + chunkLen = left; + } + encryptedStream.write(testFileContents, offset, chunkLen); + offset += chunkLen; + Assert.assertEquals("Position", offset, encryptedStream.getPos()); + left -= chunkLen; + } + encryptedStream.close(); + + AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key); + SeekableInputStream decryptedStream = decryptedFile.newStream(); + Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); + + byte[] chunk = new byte[testFileSize]; + + for (int n = 0; n < 100; n++) { + int chunkLen = random.nextInt(testFileSize); + int pos = random.nextInt(testFileSize); + left = testFileSize - pos; + if (left < chunkLen) { + chunkLen = left; + } + + decryptedStream.seek(pos); + int len = decryptedStream.read(chunk, 0, chunkLen); + Assert.assertEquals("Read length", len, chunkLen); + Assert.assertEquals("Position", pos + len, decryptedStream.getPos()); + + ByteBuffer bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); + ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, pos, chunkLen); + Assert.assertEquals("Read contents", bb1, bb2); + } + decryptedStream.close(); + } + } + } + + @Test + public void testAlignedWriteRead() throws IOException { + Random random = new Random(); + int[] testFileSizes = {AesGcmOutputStream.plainBlockSize, AesGcmOutputStream.plainBlockSize + 1, + AesGcmOutputStream.plainBlockSize - 1}; + for (int testFileSize : testFileSizes) { + byte[] testFileContents = new byte[testFileSize]; + random.nextBytes(testFileContents); + byte[] key = new byte[16]; random.nextBytes(key); - File testFile = temp.newFile(); + File testFile = temp.newFile(); AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key); PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); - int maxChunkLen = testFileSize / 5; int offset = 0; + int chunkLen = AesGcmOutputStream.plainBlockSize; int left = testFileSize; while (left > 0) { - int chunkLen = random.nextInt(maxChunkLen); if (chunkLen > left) { chunkLen = left; } @@ -65,37 +144,34 @@ public void testRandomWriteRead() throws IOException { offset += chunkLen; Assert.assertEquals("Position", offset, encryptedStream.getPos()); left -= chunkLen; - - if (left == 0) { - break; - } } encryptedStream.close(); AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key); SeekableInputStream decryptedStream = decryptedFile.newStream(); - Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); - byte[] chunk = new byte[testFileSize]; + offset = 0; + chunkLen = AesGcmOutputStream.plainBlockSize; + byte[] chunk = new byte[chunkLen]; + left = testFileSize; - for (int n = 0; n < 1000; n++) { - int chunkLen = random.nextInt(testFileSize); - int pos = random.nextInt(testFileSize); - left = testFileSize - pos; - if (left < chunkLen) { + while (left > 0) { + if (chunkLen > left) { chunkLen = left; } - decryptedStream.seek(pos); + decryptedStream.seek(offset); int len = decryptedStream.read(chunk, 0, chunkLen); - Assert.assertEquals("Read length", len, chunkLen); + Assert.assertEquals("Position", offset + len, decryptedStream.getPos()); ByteBuffer bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); - ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, pos, chunkLen); - + ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, offset, chunkLen); Assert.assertEquals("Read contents", bb1, bb2); + + offset += len; + left = testFileSize - offset; } decryptedStream.close(); From 936478884ed2e652d72038fad765cb02023010c2 Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Thu, 23 Jun 2022 15:23:49 +0300 Subject: [PATCH 04/16] build fix --- .../java/org/apache/iceberg/encryption/AesGcmInputStream.java | 1 - 1 file changed, 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index b6f750c2c7a4..08b9cf3dc174 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -20,7 +20,6 @@ package org.apache.iceberg.encryption; import java.io.IOException; -import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; From 07579ee4c1fbe46fb21eea0b52232e12a1765b89 Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Tue, 12 Jul 2022 14:51:17 +0300 Subject: [PATCH 05/16] optimize skip --- .../iceberg/encryption/AesGcmInputStream.java | 29 +++++++++++++++++++ .../iceberg/encryption/TestGcmStreams.java | 26 ++++++++++++++++- 2 files changed, 54 insertions(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index 08b9cf3dc174..a3342c73ede8 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -20,6 +20,7 @@ package org.apache.iceberg.encryption; import java.io.IOException; +import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; @@ -172,6 +173,34 @@ public void seek(long newPos) throws IOException { plainStreamPosition = newPos; } + @Override + public long skip(long n) { + if (n <= 0) { + return 0; + } + if (plainStreamPosition == plainStreamSize) { + return 0; + } + + long newPosition = plainStreamPosition + n; + if (newPosition > plainStreamSize) { + long skipped = plainStreamSize - plainStreamPosition; + try { + seek(plainStreamSize); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return skipped; + } + + try { + seek(newPosition); + } catch (IOException e) { + throw new UncheckedIOException(e); + } + return n; + } + @Override public long getPos() throws IOException { return plainStreamPosition; diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java index 76171c60c33e..248fd2a33c78 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -95,6 +95,7 @@ public void testRandomWriteRead() throws IOException { byte[] chunk = new byte[testFileSize]; + // Test seek and read for (int n = 0; n < 100; n++) { int chunkLen = random.nextInt(testFileSize); int pos = random.nextInt(testFileSize); @@ -106,12 +107,35 @@ public void testRandomWriteRead() throws IOException { decryptedStream.seek(pos); int len = decryptedStream.read(chunk, 0, chunkLen); Assert.assertEquals("Read length", len, chunkLen); - Assert.assertEquals("Position", pos + len, decryptedStream.getPos()); + long pos2 = decryptedStream.getPos(); + Assert.assertEquals("Position", pos + len, pos2); ByteBuffer bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); ByteBuffer bb2 = ByteBuffer.wrap(testFileContents, pos, chunkLen); Assert.assertEquals("Read contents", bb1, bb2); + + // Test skip + long toSkip = random.nextInt(testFileSize); + long skipped = decryptedStream.skip(toSkip); + if (pos2 + toSkip < testFileSize) { + Assert.assertEquals("Skipped", toSkip, skipped); + } else { + Assert.assertEquals("Skipped", (testFileSize - pos2), skipped); + } + int pos3 = (int) decryptedStream.getPos(); + Assert.assertEquals("Position", pos2 + skipped, pos3); + + chunkLen = random.nextInt(testFileSize); + left = testFileSize - pos3; + if (left < chunkLen) { + chunkLen = left; + } + decryptedStream.read(chunk, 0, chunkLen); + bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); + bb2 = ByteBuffer.wrap(testFileContents, pos3, chunkLen); + Assert.assertEquals("Read contents", bb1, bb2); } + decryptedStream.close(); } } From 16192f5e572f4472838bb2f37f15658d604a848d Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Mon, 17 Jul 2023 15:12:09 +0300 Subject: [PATCH 06/16] address review 3. Also, resolve stale collision. --- .../iceberg/encryption/AesGcmInputFile.java | 22 ++- .../iceberg/encryption/AesGcmInputStream.java | 176 +++++++++++------- .../iceberg/encryption/AesGcmOutputFile.java | 12 +- .../encryption/AesGcmOutputStream.java | 32 ++-- .../apache/iceberg/encryption/Ciphers.java | 38 +++- 5 files changed, 183 insertions(+), 97 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java index 25208da5e41f..555c6aef099e 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg.encryption; import java.io.IOException; @@ -27,35 +26,40 @@ public class AesGcmInputFile implements InputFile { private final InputFile sourceFile; private final byte[] dataKey; + private final byte[] fileAADPrefix; private long plaintextLength; - public AesGcmInputFile(InputFile sourceFile, byte[] dataKey) { + public AesGcmInputFile(InputFile sourceFile, byte[] dataKey, byte[] fileAADPrefix) { this.sourceFile = sourceFile; this.dataKey = dataKey; + this.fileAADPrefix = fileAADPrefix; this.plaintextLength = -1; } @Override public long getLength() { if (plaintextLength == -1) { - try { - this.newStream().close(); - } catch (IOException e) { - throw new UncheckedIOException(e); - } + // Presumes all streams use hard-coded plaintext block size. + // Actual plaintext block size is checked upon stream creation (exception if different). + plaintextLength = + AesGcmInputStream.calculatePlaintextLength( + sourceFile.getLength(), AesGcmOutputStream.plainBlockSize); } return plaintextLength; } @Override public SeekableInputStream newStream() { + getLength(); AesGcmInputStream result; try { - result = new AesGcmInputStream(sourceFile.newStream(), sourceFile.getLength(), dataKey, null); + result = + new AesGcmInputStream( + sourceFile.newStream(), sourceFile.getLength(), dataKey, fileAADPrefix); } catch (IOException e) { throw new UncheckedIOException(e); } - plaintextLength = result.plaintextStreamSize(); + return result; } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index a3342c73ede8..a7ab81d179c4 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -16,91 +16,94 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg.encryption; +import java.io.EOFException; import java.io.IOException; import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; import java.util.Arrays; +import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class AesGcmInputStream extends SeekableInputStream { private final SeekableInputStream sourceStream; - private final boolean emptyCipherStream; - private final long netSourceFileSize; private final Ciphers.AesGcmDecryptor gcmDecryptor; - private final byte[] ciphertextBlockBuffer; + private final byte[] cipherBlockBuffer; private final int cipherBlockSize; private final int plainBlockSize; private final int numberOfBlocks; private final int lastCipherBlockSize; private final long plainStreamSize; - private final byte[] fileAadPrefix; + private final byte[] fileAADPrefix; private long plainStreamPosition; private int currentBlockIndex; private int currentOffsetInPlainBlock; + private byte[] currentDecryptedBlock; + private int currentDecryptedBlockIndex; - AesGcmInputStream(SeekableInputStream sourceStream, long sourceLength, - byte[] aesKey, byte[] fileAadPrefix) throws IOException { - this.netSourceFileSize = sourceLength - Ciphers.GCM_STREAM_PREFIX_LENGTH; - Preconditions.checkArgument(netSourceFileSize >= 0, - "Source length " + sourceLength + " is shorter than GCM prefix. File is not encrypted"); - - this.emptyCipherStream = (0 == netSourceFileSize); + AesGcmInputStream( + SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix) + throws IOException { + long netSourceLength = netSourceFileLength(sourceLength); + boolean emptyCipherStream = (0 == netSourceLength); this.sourceStream = sourceStream; - byte[] prefixBytes = new byte[Ciphers.GCM_STREAM_PREFIX_LENGTH]; - int fetched = sourceStream.read(prefixBytes); - Preconditions.checkState(fetched == Ciphers.GCM_STREAM_PREFIX_LENGTH, - "Insufficient read " + fetched + - ". The stream length should be at least " + Ciphers.GCM_STREAM_PREFIX_LENGTH); - + byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH]; + IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length); byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length]; - System.arraycopy(prefixBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length); - Preconditions.checkState(Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic), - "Cannot open encrypted file, it does not begin with magic string " + Ciphers.GCM_STREAM_MAGIC_STRING); + System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length); + Preconditions.checkState( + Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic), + "Cannot open encrypted file, it does not begin with magic string " + + Ciphers.GCM_STREAM_MAGIC_STRING); + this.currentDecryptedBlockIndex = -1; if (!emptyCipherStream) { this.plainStreamPosition = 0; - this.fileAadPrefix = fileAadPrefix; + this.fileAADPrefix = fileAADPrefix; gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey); - plainBlockSize = ByteBuffer.wrap(prefixBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4) - .order(ByteOrder.LITTLE_ENDIAN).getInt(); + plainBlockSize = + ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4) + .order(ByteOrder.LITTLE_ENDIAN) + .getInt(); Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize); + Preconditions.checkState( + plainBlockSize == AesGcmOutputStream.plainBlockSize, + "Wrong plainBlockSize " + + plainBlockSize + + ". Only size of " + + AesGcmOutputStream.plainBlockSize + + " is currently supported"); + cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH; - this.ciphertextBlockBuffer = new byte[cipherBlockSize]; + this.cipherBlockBuffer = new byte[cipherBlockSize]; this.currentBlockIndex = 0; this.currentOffsetInPlainBlock = 0; - int numberOfFullBlocks = Math.toIntExact(netSourceFileSize / cipherBlockSize); - int cipherBytesInLastBlock = Math.toIntExact(netSourceFileSize - numberOfFullBlocks * cipherBlockSize); + int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize); + int cipherBytesInLastBlock = + Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize); boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1; lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0 - int plainBytesInLastBlock = fullBlocksOnly ? 0 : - (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH); - plainStreamSize = numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock; + plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize); } else { plainStreamSize = 0; gcmDecryptor = null; - ciphertextBlockBuffer = null; + cipherBlockBuffer = null; cipherBlockSize = -1; plainBlockSize = -1; numberOfBlocks = -1; lastCipherBlockSize = -1; - this.fileAadPrefix = null; + this.fileAADPrefix = null; } } - public long plaintextStreamSize() { - return plainStreamSize; - } - @Override public int available() throws IOException { long maxAvailable = plainStreamSize - plainStreamPosition; @@ -114,51 +117,48 @@ public int available() throws IOException { @Override public int read(byte[] b, int off, int len) throws IOException { - if (len < 0) { - throw new IOException("Negative read length " + len); + Preconditions.checkState(len >= 0, "Negative read length " + len); + + if (available() <= 0 && len > 0) { + throw new EOFException(); } - if (available() <= 0) { - return -1; + if (len == 0) { + return 0; } - boolean lastBlock = (currentBlockIndex + 1 == numberOfBlocks); + boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks); int resultBufferOffset = off; - int remaining = len; - - sourceStream.seek(Ciphers.GCM_STREAM_PREFIX_LENGTH + currentBlockIndex * cipherBlockSize); + int remainingBytesToRead = len; - while (remaining > 0) { - int toLoad = lastBlock ? lastCipherBlockSize : cipherBlockSize; - int loaded = sourceStream.read(ciphertextBlockBuffer, 0, toLoad); - if (loaded != toLoad) { - throw new IOException("Should read " + toLoad + " bytes, but got only " + loaded + " bytes"); - } + if (currentBlockIndex != currentDecryptedBlockIndex) { + sourceStream.seek(blockOffset(currentBlockIndex)); + } - byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex); - byte[] plaintextBlock = gcmDecryptor.decrypt(ciphertextBlockBuffer, 0, toLoad, aad); + while (remainingBytesToRead > 0) { + byte[] plainBlock = decryptNextBlock(isLastBlockInStream); - int remainingInBlock = plaintextBlock.length - currentOffsetInPlainBlock; - boolean finishTheBlock = remaining >= remainingInBlock; - int toCopy = finishTheBlock ? remainingInBlock : remaining; + int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock; + boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock; + int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead; + System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy); + remainingBytesToRead -= bytesToCopy; + resultBufferOffset += bytesToCopy; + currentOffsetInPlainBlock += bytesToCopy; - System.arraycopy(plaintextBlock, currentOffsetInPlainBlock, b, resultBufferOffset, toCopy); - remaining -= toCopy; - resultBufferOffset += toCopy; - currentOffsetInPlainBlock += toCopy; - boolean endOfStream = lastBlock && finishTheBlock; + boolean endOfStream = isLastBlockInStream && finishTheBlock; if (endOfStream) { break; } if (finishTheBlock) { currentBlockIndex++; currentOffsetInPlainBlock = 0; - lastBlock = (currentBlockIndex + 1 == numberOfBlocks); + isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks); } } - plainStreamPosition += len - remaining; - return len - remaining; + plainStreamPosition += len - remainingBytesToRead; + return len - remainingBytesToRead; } @Override @@ -166,7 +166,8 @@ public void seek(long newPos) throws IOException { if (newPos < 0) { throw new IOException("Negative new position " + newPos); } else if (newPos > plainStreamSize) { - throw new IOException("New position " + newPos + " exceeds the max stream size " + plainStreamSize); + throw new IOException( + "New position " + newPos + " exceeds the max stream size " + plainStreamSize); } currentBlockIndex = Math.toIntExact(newPos / plainBlockSize); currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize); @@ -214,5 +215,52 @@ public int read() throws IOException { @Override public void close() throws IOException { sourceStream.close(); + currentDecryptedBlock = null; + } + + static long calculatePlaintextLength(long sourceLength, int plainBlockSize) { + long netSourceFileLength = netSourceFileLength(sourceLength); + if (netSourceFileLength == 0) { + return 0; + } + + int cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH; + int numberOfFullBlocks = Math.toIntExact(netSourceFileLength / cipherBlockSize); + + int cipherBytesInLastBlock = + Math.toIntExact(netSourceFileLength - numberOfFullBlocks * cipherBlockSize); + boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); + int plainBytesInLastBlock = + fullBlocksOnly + ? 0 + : (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH); + return numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock; + } + + private byte[] decryptNextBlock(boolean isLastBlockInStream) throws IOException { + if (currentBlockIndex == currentDecryptedBlockIndex) { + return currentDecryptedBlock; + } + + int currentCipherBlockSize = isLastBlockInStream ? lastCipherBlockSize : cipherBlockSize; + IOUtil.readFully(sourceStream, cipherBlockBuffer, 0, currentCipherBlockSize); + + byte[] aad = Ciphers.streamBlockAAD(fileAADPrefix, currentBlockIndex); + byte[] result = gcmDecryptor.decrypt(cipherBlockBuffer, 0, currentCipherBlockSize, aad); + currentDecryptedBlockIndex = currentBlockIndex; + currentDecryptedBlock = result; + return result; + } + + private long blockOffset(int blockIndex) { + return Ciphers.GCM_STREAM_HEADER_LENGTH + blockIndex * cipherBlockSize; + } + + private static long netSourceFileLength(long sourceFileLength) { + long netSourceLength = sourceFileLength - Ciphers.GCM_STREAM_HEADER_LENGTH; + Preconditions.checkArgument( + netSourceLength >= 0, + "Source length " + sourceFileLength + " is shorter than GCM prefix. File is not encrypted"); + return netSourceLength; } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java index 865d4a0eeae0..15d3c19e56c3 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg.encryption; import java.io.IOException; @@ -28,16 +27,18 @@ public class AesGcmOutputFile implements OutputFile { private final OutputFile targetFile; private final byte[] dataKey; + private final byte[] fileAADPrefix; - public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey) { + public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey, byte[] fileAADPrefix) { this.targetFile = targetFile; this.dataKey = dataKey; + this.fileAADPrefix = fileAADPrefix; } @Override public PositionOutputStream create() { try { - return new AesGcmOutputStream(targetFile.create(), dataKey, null); + return new AesGcmOutputStream(targetFile.create(), dataKey, fileAADPrefix); } catch (IOException e) { throw new UncheckedIOException("Failed to create GCM stream for " + targetFile.location(), e); } @@ -46,9 +47,10 @@ public PositionOutputStream create() { @Override public PositionOutputStream createOrOverwrite() { try { - return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, null); + return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, fileAADPrefix); } catch (IOException e) { - throw new UncheckedIOException("Failed to create or overwrite GCM stream for " + targetFile.location(), e); + throw new UncheckedIOException( + "Failed to create or overwrite GCM stream for " + targetFile.location(), e); } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index 4b8c147af8fe..28eae02debe5 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg.encryption; import java.io.IOException; @@ -29,27 +28,30 @@ public class AesGcmOutputStream extends PositionOutputStream { private final Ciphers.AesGcmEncryptor gcmEncryptor; private final PositionOutputStream targetStream; - private final byte[] plaintextBlockBuffer; + private final byte[] plainBlockBuffer; private final byte[] fileAadPrefix; private int positionInBuffer; private long streamPosition; private int currentBlockIndex; - AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) throws IOException { + AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) + throws IOException { this.targetStream = targetStream; this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey); - this.plaintextBlockBuffer = new byte[plainBlockSize]; + this.plainBlockBuffer = new byte[plainBlockSize]; this.positionInBuffer = 0; this.streamPosition = 0; this.currentBlockIndex = 0; this.fileAadPrefix = fileAadPrefix; - byte[] prefixBytes = ByteBuffer.allocate(Ciphers.GCM_STREAM_PREFIX_LENGTH).order(ByteOrder.LITTLE_ENDIAN) - .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) - .putInt(plainBlockSize) - .array(); - targetStream.write(prefixBytes); + byte[] headerBytes = + ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH) + .order(ByteOrder.LITTLE_ENDIAN) + .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) + .putInt(plainBlockSize) + .array(); + targetStream.write(headerBytes); } @Override @@ -60,7 +62,8 @@ public void write(int b) throws IOException { @Override public void write(byte[] b, int off, int len) throws IOException { if (b.length - off < len) { - throw new IOException("Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len); + throw new IOException( + "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len); } int remaining = len; int offset = off; @@ -69,7 +72,7 @@ public void write(byte[] b, int off, int len) throws IOException { int freeBlockBytes = plainBlockSize - positionInBuffer; int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining; - System.arraycopy(b, offset, plaintextBlockBuffer, positionInBuffer, toWrite); + System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite); positionInBuffer += toWrite; if (positionInBuffer == plainBlockSize) { encryptAndWriteBlock(); @@ -101,9 +104,12 @@ public void close() throws IOException { } private void encryptAndWriteBlock() throws IOException { + if (currentBlockIndex == Integer.MAX_VALUE) { + throw new IOException("Too many blocks - exceed Integer.MAX_VALUE"); + } byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex); - byte[] cipherText = gcmEncryptor.encrypt(plaintextBlockBuffer, 0, positionInBuffer, aad); + byte[] cipherBlockBuffer = gcmEncryptor.encrypt(plainBlockBuffer, 0, positionInBuffer, aad); currentBlockIndex++; - targetStream.write(cipherText); + targetStream.write(cipherBlockBuffer); } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index 6b5436cbf2cb..dbe47068f1e2 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -18,6 +18,8 @@ */ package org.apache.iceberg.encryption; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; import java.nio.charset.StandardCharsets; import java.security.GeneralSecurityException; import java.security.SecureRandom; @@ -26,20 +28,20 @@ import javax.crypto.spec.GCMParameterSpec; import javax.crypto.spec.SecretKeySpec; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; -import org.apache.iceberg.relocated.com.google.common.primitives.Ints; public class Ciphers { public static final int NONCE_LENGTH = 12; public static final int GCM_TAG_LENGTH = 16; public static final String GCM_STREAM_MAGIC_STRING = "GCM1"; - static final byte[] GCM_STREAM_MAGIC_ARRAY = GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8); - static final int GCM_STREAM_PREFIX_LENGTH = GCM_STREAM_MAGIC_ARRAY.length + 4; // magic_len + block_size_len + static final byte[] GCM_STREAM_MAGIC_ARRAY = + GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8); + static final int GCM_STREAM_HEADER_LENGTH = + GCM_STREAM_MAGIC_ARRAY.length + 4; // magic_len + block_size_len private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH; - private Ciphers() { - } + private Ciphers() {} public static class AesGcmEncryptor { private final SecretKeySpec aesKey; @@ -114,6 +116,7 @@ public AesGcmDecryptor(byte[] keyBytes) { } } +<<<<<<< HEAD <<<<<<< HEAD public byte[] decrypt(byte[] ciphertext, byte[] aad) { int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH; @@ -134,6 +137,20 @@ public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLen " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" + " with AES GCM cipher"); >>>>>>> 52b944a45 (address review 2) +======= + public byte[] decrypt(byte[] ciphertext, byte[] aad) { + return decrypt(ciphertext, 0, ciphertext.length, aad); + } + + public byte[] decrypt( + byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) { + Preconditions.checkState( + ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1, + "Cannot decrypt cipher text of length " + + ciphertext.length + + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" + + " with AES GCM cipher"); +>>>>>>> 22e48b448 (address review 3) // Get the nonce from ciphertext byte[] nonce = new byte[NONCE_LENGTH]; @@ -155,10 +172,18 @@ public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLen e); ======= return cipher.doFinal(ciphertext, ciphertextOffset + NONCE_LENGTH, inputLength); +<<<<<<< HEAD } catch (AEADBadTagException e) { throw new RuntimeException("GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" + "data. AES GCM doesn't differentiate between these two.", e); >>>>>>> 52b944a45 (address review 2) +======= + } catch (AEADBadTagException e) { + throw new RuntimeException( + "GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" + + "data. AES GCM doesn't differentiate between these two.", + e); +>>>>>>> 22e48b448 (address review 3) } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to decrypt", e); } @@ -166,7 +191,8 @@ public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLen } static byte[] streamBlockAAD(byte[] fileAadPrefix, int currentBlockIndex) { - byte[] blockAAD = Ints.toByteArray(currentBlockIndex); + byte[] blockAAD = + ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN).putInt(currentBlockIndex).array(); if (null == fileAadPrefix) { return blockAAD; } else { From 23567e0190a81d37b2ccb610858086dbb9ba362a Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Mon, 17 Jul 2023 15:35:26 +0300 Subject: [PATCH 07/16] code format, unitest fix, long math, revapi, magic string fix unitest, long math, revapi unitest code format remove util constructor cast to long Utility class constructor private constructor fix revapi optimize stream seeks update magic string --- .palantir/revapi.yml | 6 ++ .../iceberg/encryption/AesGcmInputFile.java | 2 + .../iceberg/encryption/AesGcmInputStream.java | 22 +++++--- .../encryption/AesGcmOutputStream.java | 1 + .../apache/iceberg/encryption/Ciphers.java | 41 +------------- .../iceberg/encryption/TestGcmStreams.java | 55 ++++++++++++++----- 6 files changed, 68 insertions(+), 59 deletions(-) diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index 4bfde0b516c1..e8e04bf8dc94 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -743,6 +743,12 @@ acceptedBreaks: new: "method java.util.List org.apache.iceberg.rest.requests.UpdateTableRequest::requirements()" justification: "Signature changed to an interface, but this is safe because\ \ of type erasure and the original type is always returned" + "1.3.0": + org.apache.iceberg:iceberg-core: + - code: "java.method.visibilityReduced" + old: "method void org.apache.iceberg.encryption.Ciphers::()" + new: "method void org.apache.iceberg.encryption.Ciphers::()" + justification: "Static utility class - should not have public constructor" apache-iceberg-0.14.0: org.apache.iceberg:iceberg-api: - code: "java.class.defaultSerializationChanged" diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java index 555c6aef099e..8663cd8745da 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java @@ -45,6 +45,7 @@ public long getLength() { AesGcmInputStream.calculatePlaintextLength( sourceFile.getLength(), AesGcmOutputStream.plainBlockSize); } + return plaintextLength; } @@ -52,6 +53,7 @@ public long getLength() { public SeekableInputStream newStream() { getLength(); AesGcmInputStream result; + try { result = new AesGcmInputStream( diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index a7ab81d179c4..19aa05cf4de6 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -131,10 +131,6 @@ public int read(byte[] b, int off, int len) throws IOException { int resultBufferOffset = off; int remainingBytesToRead = len; - if (currentBlockIndex != currentDecryptedBlockIndex) { - sourceStream.seek(blockOffset(currentBlockIndex)); - } - while (remainingBytesToRead > 0) { byte[] plainBlock = decryptNextBlock(isLastBlockInStream); @@ -147,9 +143,11 @@ public int read(byte[] b, int off, int len) throws IOException { currentOffsetInPlainBlock += bytesToCopy; boolean endOfStream = isLastBlockInStream && finishTheBlock; + if (endOfStream) { break; } + if (finishTheBlock) { currentBlockIndex++; currentOffsetInPlainBlock = 0; @@ -169,6 +167,7 @@ public void seek(long newPos) throws IOException { throw new IOException( "New position " + newPos + " exceeds the max stream size " + plainStreamSize); } + currentBlockIndex = Math.toIntExact(newPos / plainBlockSize); currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize); plainStreamPosition = newPos; @@ -179,11 +178,13 @@ public long skip(long n) { if (n <= 0) { return 0; } + if (plainStreamPosition == plainStreamSize) { return 0; } long newPosition = plainStreamPosition + n; + if (newPosition > plainStreamSize) { long skipped = plainStreamSize - plainStreamPosition; try { @@ -220,13 +221,13 @@ public void close() throws IOException { static long calculatePlaintextLength(long sourceLength, int plainBlockSize) { long netSourceFileLength = netSourceFileLength(sourceLength); + if (netSourceFileLength == 0) { return 0; } int cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH; int numberOfFullBlocks = Math.toIntExact(netSourceFileLength / cipherBlockSize); - int cipherBytesInLastBlock = Math.toIntExact(netSourceFileLength - numberOfFullBlocks * cipherBlockSize); boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); @@ -234,7 +235,8 @@ static long calculatePlaintextLength(long sourceLength, int plainBlockSize) { fullBlocksOnly ? 0 : (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH); - return numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock; + + return (long) numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock; } private byte[] decryptNextBlock(boolean isLastBlockInStream) throws IOException { @@ -242,6 +244,11 @@ private byte[] decryptNextBlock(boolean isLastBlockInStream) throws IOException return currentDecryptedBlock; } + long blockPositionInStream = blockOffset(currentBlockIndex); + if (sourceStream.getPos() != blockPositionInStream) { + sourceStream.seek(blockPositionInStream); + } + int currentCipherBlockSize = isLastBlockInStream ? lastCipherBlockSize : cipherBlockSize; IOUtil.readFully(sourceStream, cipherBlockBuffer, 0, currentCipherBlockSize); @@ -253,7 +260,7 @@ private byte[] decryptNextBlock(boolean isLastBlockInStream) throws IOException } private long blockOffset(int blockIndex) { - return Ciphers.GCM_STREAM_HEADER_LENGTH + blockIndex * cipherBlockSize; + return (long) blockIndex * cipherBlockSize + Ciphers.GCM_STREAM_HEADER_LENGTH; } private static long netSourceFileLength(long sourceFileLength) { @@ -261,6 +268,7 @@ private static long netSourceFileLength(long sourceFileLength) { Preconditions.checkArgument( netSourceLength >= 0, "Source length " + sourceFileLength + " is shorter than GCM prefix. File is not encrypted"); + return netSourceLength; } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index 28eae02debe5..ce165c39c70f 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -107,6 +107,7 @@ private void encryptAndWriteBlock() throws IOException { if (currentBlockIndex == Integer.MAX_VALUE) { throw new IOException("Too many blocks - exceed Integer.MAX_VALUE"); } + byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex); byte[] cipherBlockBuffer = gcmEncryptor.encrypt(plainBlockBuffer, 0, positionInBuffer, aad); currentBlockIndex++; diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index dbe47068f1e2..cf43880a7475 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -32,7 +32,7 @@ public class Ciphers { public static final int NONCE_LENGTH = 12; public static final int GCM_TAG_LENGTH = 16; - public static final String GCM_STREAM_MAGIC_STRING = "GCM1"; + public static final String GCM_STREAM_MAGIC_STRING = "AGS1"; static final byte[] GCM_STREAM_MAGIC_ARRAY = GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8); @@ -116,28 +116,6 @@ public AesGcmDecryptor(byte[] keyBytes) { } } -<<<<<<< HEAD -<<<<<<< HEAD - 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"); -======= - public byte[] decrypt(byte[] ciphertext, byte[] aad) { - return decrypt(ciphertext, 0, ciphertext.length, aad); - } - - public byte[] decrypt(byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) { - Preconditions.checkState(ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1, - "Cannot decrypt cipher text of length " + ciphertext.length + - " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" + - " with AES GCM cipher"); ->>>>>>> 52b944a45 (address review 2) -======= public byte[] decrypt(byte[] ciphertext, byte[] aad) { return decrypt(ciphertext, 0, ciphertext.length, aad); } @@ -150,7 +128,6 @@ public byte[] decrypt( + ciphertext.length + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" + " with AES GCM cipher"); ->>>>>>> 22e48b448 (address review 3) // Get the nonce from ciphertext byte[] nonce = new byte[NONCE_LENGTH]; @@ -163,27 +140,12 @@ public byte[] decrypt( if (null != aad) { cipher.updateAAD(aad); } -<<<<<<< HEAD - cipher.doFinal(ciphertext, NONCE_LENGTH, inputLength, plainText, 0); - } catch (AEADBadTagException e) { - throw new RuntimeException( - "GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" - + "data. AES GCM doesn't differentiate between these two.. ", - e); -======= return cipher.doFinal(ciphertext, ciphertextOffset + NONCE_LENGTH, inputLength); -<<<<<<< HEAD - } catch (AEADBadTagException e) { - throw new RuntimeException("GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" + - "data. AES GCM doesn't differentiate between these two.", e); ->>>>>>> 52b944a45 (address review 2) -======= } catch (AEADBadTagException e) { throw new RuntimeException( "GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" + "data. AES GCM doesn't differentiate between these two.", e); ->>>>>>> 22e48b448 (address review 3) } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to decrypt", e); } @@ -193,6 +155,7 @@ public byte[] decrypt( static byte[] streamBlockAAD(byte[] fileAadPrefix, int currentBlockIndex) { byte[] blockAAD = ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN).putInt(currentBlockIndex).array(); + if (null == fileAadPrefix) { return blockAAD; } else { diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java index 248fd2a33c78..d6905f1306f5 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -16,7 +16,6 @@ * specific language governing permissions and limitations * under the License. */ - package org.apache.iceberg.encryption; import java.io.File; @@ -33,20 +32,22 @@ public class TestGcmStreams { - @Rule - public TemporaryFolder temp = new TemporaryFolder(); + @Rule public TemporaryFolder temp = new TemporaryFolder(); @Test public void testEmptyFile() throws IOException { Random random = new Random(); byte[] key = new byte[16]; random.nextBytes(key); + byte[] aadPrefix = new byte[16]; + random.nextBytes(aadPrefix); File testFile = temp.newFile(); - AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key); + AesGcmOutputFile encryptedFile = + new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); encryptedStream.close(); - AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key); + AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); SeekableInputStream decryptedStream = decryptedFile.newStream(); Assert.assertEquals("File size", 0, decryptedFile.getLength()); @@ -59,18 +60,27 @@ public void testRandomWriteRead() throws IOException { int smallerThanBlock = (int) (AesGcmOutputStream.plainBlockSize * 0.5); int largerThanBlock = (int) (AesGcmOutputStream.plainBlockSize * 1.5); int alignedWithBlock = AesGcmOutputStream.plainBlockSize; - int[] testFileSizes = {smallerThanBlock, largerThanBlock, alignedWithBlock, alignedWithBlock - 1, - alignedWithBlock + 1}; + int[] testFileSizes = { + smallerThanBlock, + largerThanBlock, + alignedWithBlock, + alignedWithBlock - 1, + alignedWithBlock + 1 + }; + for (int testFileSize : testFileSizes) { byte[] testFileContents = new byte[testFileSize]; random.nextBytes(testFileContents); int[] aesKeyLengthArray = {16, 24, 32}; + byte[] aadPrefix = new byte[16]; for (int keyLength : aesKeyLengthArray) { byte[] key = new byte[keyLength]; random.nextBytes(key); + random.nextBytes(aadPrefix); File testFile = temp.newFile(); - AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key); + AesGcmOutputFile encryptedFile = + new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); int maxChunkLen = testFileSize / 5; @@ -87,9 +97,11 @@ public void testRandomWriteRead() throws IOException { Assert.assertEquals("Position", offset, encryptedStream.getPos()); left -= chunkLen; } + encryptedStream.close(); - AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key); + AesGcmInputFile decryptedFile = + new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); SeekableInputStream decryptedStream = decryptedFile.newStream(); Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); @@ -100,6 +112,7 @@ public void testRandomWriteRead() throws IOException { int chunkLen = random.nextInt(testFileSize); int pos = random.nextInt(testFileSize); left = testFileSize - pos; + if (left < chunkLen) { chunkLen = left; } @@ -117,19 +130,23 @@ public void testRandomWriteRead() throws IOException { // Test skip long toSkip = random.nextInt(testFileSize); long skipped = decryptedStream.skip(toSkip); + if (pos2 + toSkip < testFileSize) { Assert.assertEquals("Skipped", toSkip, skipped); } else { Assert.assertEquals("Skipped", (testFileSize - pos2), skipped); } + int pos3 = (int) decryptedStream.getPos(); Assert.assertEquals("Position", pos2 + skipped, pos3); chunkLen = random.nextInt(testFileSize); left = testFileSize - pos3; + if (left < chunkLen) { chunkLen = left; } + decryptedStream.read(chunk, 0, chunkLen); bb1 = ByteBuffer.wrap(chunk, 0, chunkLen); bb2 = ByteBuffer.wrap(testFileContents, pos3, chunkLen); @@ -144,16 +161,23 @@ public void testRandomWriteRead() throws IOException { @Test public void testAlignedWriteRead() throws IOException { Random random = new Random(); - int[] testFileSizes = {AesGcmOutputStream.plainBlockSize, AesGcmOutputStream.plainBlockSize + 1, - AesGcmOutputStream.plainBlockSize - 1}; + int[] testFileSizes = { + AesGcmOutputStream.plainBlockSize, + AesGcmOutputStream.plainBlockSize + 1, + AesGcmOutputStream.plainBlockSize - 1 + }; + for (int testFileSize : testFileSizes) { byte[] testFileContents = new byte[testFileSize]; random.nextBytes(testFileContents); byte[] key = new byte[16]; random.nextBytes(key); + byte[] aadPrefix = new byte[16]; + random.nextBytes(aadPrefix); File testFile = temp.newFile(); - AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key); + AesGcmOutputFile encryptedFile = + new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); int offset = 0; @@ -161,17 +185,21 @@ public void testAlignedWriteRead() throws IOException { int left = testFileSize; while (left > 0) { + if (chunkLen > left) { chunkLen = left; } + encryptedStream.write(testFileContents, offset, chunkLen); offset += chunkLen; Assert.assertEquals("Position", offset, encryptedStream.getPos()); left -= chunkLen; } + encryptedStream.close(); - AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key); + AesGcmInputFile decryptedFile = + new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); SeekableInputStream decryptedStream = decryptedFile.newStream(); Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); @@ -181,6 +209,7 @@ public void testAlignedWriteRead() throws IOException { left = testFileSize; while (left > 0) { + if (chunkLen > left) { chunkLen = left; } From 08983e10a9d272ca7558541f6cfd3cf0c4de8d5b Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Thu, 20 Jul 2023 10:41:36 +0300 Subject: [PATCH 08/16] EOFException --- .../java/org/apache/iceberg/encryption/AesGcmInputStream.java | 2 +- 1 file changed, 1 insertion(+), 1 deletion(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index 19aa05cf4de6..d5cae583cfba 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -164,7 +164,7 @@ public void seek(long newPos) throws IOException { if (newPos < 0) { throw new IOException("Negative new position " + newPos); } else if (newPos > plainStreamSize) { - throw new IOException( + throw new EOFException( "New position " + newPos + " exceeds the max stream size " + plainStreamSize); } From 344529745639a306b3aff635ddac8ff6a70bd399 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Sun, 30 Jul 2023 14:42:26 -0700 Subject: [PATCH 09/16] Refactor AesGcmInputStream. --- .../iceberg/encryption/AesGcmInputFile.java | 26 +- .../iceberg/encryption/AesGcmInputStream.java | 278 ++++++++---------- .../encryption/AesGcmOutputStream.java | 9 +- .../apache/iceberg/encryption/Ciphers.java | 4 + .../iceberg/encryption/TestGcmStreams.java | 16 +- 5 files changed, 153 insertions(+), 180 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java index 8663cd8745da..ab0e91bcbd61 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java @@ -18,10 +18,9 @@ */ package org.apache.iceberg.encryption; -import java.io.IOException; -import java.io.UncheckedIOException; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class AesGcmInputFile implements InputFile { private final InputFile sourceFile; @@ -40,10 +39,7 @@ public AesGcmInputFile(InputFile sourceFile, byte[] dataKey, byte[] fileAADPrefi public long getLength() { if (plaintextLength == -1) { // Presumes all streams use hard-coded plaintext block size. - // Actual plaintext block size is checked upon stream creation (exception if different). - plaintextLength = - AesGcmInputStream.calculatePlaintextLength( - sourceFile.getLength(), AesGcmOutputStream.plainBlockSize); + plaintextLength = AesGcmInputStream.calculatePlaintextLength(sourceFile.getLength()); } return plaintextLength; @@ -51,18 +47,12 @@ public long getLength() { @Override public SeekableInputStream newStream() { - getLength(); - AesGcmInputStream result; - - try { - result = - new AesGcmInputStream( - sourceFile.newStream(), sourceFile.getLength(), dataKey, fileAADPrefix); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - - return result; + long ciphertextLength = sourceFile.getLength(); + Preconditions.checkState( + ciphertextLength >= Ciphers.GCM_STREAM_HEADER_LENGTH, + "Invalid encrypted stream: %d is shorter than the GCM stream header length", + ciphertextLength); + return new AesGcmInputStream(sourceFile.newStream(), ciphertextLength, dataKey, fileAADPrefix); } @Override diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index d5cae583cfba..0bbb8ad4556e 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -20,92 +20,68 @@ import java.io.EOFException; import java.io.IOException; -import java.io.UncheckedIOException; import java.nio.ByteBuffer; import java.nio.ByteOrder; -import java.util.Arrays; import org.apache.iceberg.io.IOUtil; import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class AesGcmInputStream extends SeekableInputStream { private final SeekableInputStream sourceStream; - private final Ciphers.AesGcmDecryptor gcmDecryptor; + private final byte[] fileAADPrefix; + private final Ciphers.AesGcmDecryptor decryptor; private final byte[] cipherBlockBuffer; - private final int cipherBlockSize; - private final int plainBlockSize; - private final int numberOfBlocks; + private final long numBlocks; private final int lastCipherBlockSize; private final long plainStreamSize; - private final byte[] fileAADPrefix; private long plainStreamPosition; - private int currentBlockIndex; - private int currentOffsetInPlainBlock; - private byte[] currentDecryptedBlock; - private int currentDecryptedBlockIndex; + private long currentPlainBlockIndex; + private byte[] currentPlainBlock; + private int currentPlainBlockSize; AesGcmInputStream( - SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix) - throws IOException { - long netSourceLength = netSourceFileLength(sourceLength); - boolean emptyCipherStream = (0 == netSourceLength); + SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix) { this.sourceStream = sourceStream; + this.fileAADPrefix = fileAADPrefix; + this.decryptor = new Ciphers.AesGcmDecryptor(aesKey); + this.cipherBlockBuffer = new byte[Ciphers.CIPHER_BLOCK_SIZE]; + + this.plainStreamPosition = 0; + this.currentPlainBlockIndex = -1; + this.currentPlainBlock = null; + this.currentPlainBlockSize = 0; + + long streamLength = sourceLength - Ciphers.GCM_STREAM_HEADER_LENGTH; + long numFullBlocks = Math.toIntExact(streamLength / Ciphers.CIPHER_BLOCK_SIZE); + long cipherFullBlockLength = numFullBlocks * Ciphers.CIPHER_BLOCK_SIZE; + int cipherBytesInLastBlock = Math.toIntExact(streamLength - cipherFullBlockLength); + boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); + this.numBlocks = fullBlocksOnly ? numFullBlocks : numFullBlocks + 1; + this.lastCipherBlockSize = fullBlocksOnly ? Ciphers.CIPHER_BLOCK_SIZE : cipherBytesInLastBlock; // never 0 + + long lastPlainBlockSize = lastCipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH; + this.plainStreamSize = numFullBlocks * Ciphers.PLAIN_BLOCK_SIZE + (fullBlocksOnly ? 0 : lastPlainBlockSize); + } + + private void validateHeader() throws IOException { byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH]; IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length); - byte[] magic = new byte[Ciphers.GCM_STREAM_MAGIC_ARRAY.length]; - System.arraycopy(headerBytes, 0, magic, 0, Ciphers.GCM_STREAM_MAGIC_ARRAY.length); + Preconditions.checkState( - Arrays.equals(Ciphers.GCM_STREAM_MAGIC_ARRAY, magic), - "Cannot open encrypted file, it does not begin with magic string " - + Ciphers.GCM_STREAM_MAGIC_STRING); - this.currentDecryptedBlockIndex = -1; - - if (!emptyCipherStream) { - this.plainStreamPosition = 0; - this.fileAADPrefix = fileAADPrefix; - gcmDecryptor = new Ciphers.AesGcmDecryptor(aesKey); - plainBlockSize = - ByteBuffer.wrap(headerBytes, Ciphers.GCM_STREAM_MAGIC_ARRAY.length, 4) - .order(ByteOrder.LITTLE_ENDIAN) - .getInt(); - Preconditions.checkState(plainBlockSize > 0, "Wrong plainBlockSize " + plainBlockSize); - - Preconditions.checkState( - plainBlockSize == AesGcmOutputStream.plainBlockSize, - "Wrong plainBlockSize " - + plainBlockSize - + ". Only size of " - + AesGcmOutputStream.plainBlockSize - + " is currently supported"); - - cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH; - this.cipherBlockBuffer = new byte[cipherBlockSize]; - this.currentBlockIndex = 0; - this.currentOffsetInPlainBlock = 0; - - int numberOfFullBlocks = Math.toIntExact(netSourceLength / cipherBlockSize); - int cipherBytesInLastBlock = - Math.toIntExact(netSourceLength - numberOfFullBlocks * cipherBlockSize); - boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); - numberOfBlocks = fullBlocksOnly ? numberOfFullBlocks : numberOfFullBlocks + 1; - lastCipherBlockSize = fullBlocksOnly ? cipherBlockSize : cipherBytesInLastBlock; // never 0 - plainStreamSize = calculatePlaintextLength(sourceLength, plainBlockSize); - } else { - plainStreamSize = 0; - - gcmDecryptor = null; - cipherBlockBuffer = null; - cipherBlockSize = -1; - plainBlockSize = -1; - numberOfBlocks = -1; - lastCipherBlockSize = -1; - this.fileAADPrefix = null; - } + Ciphers.GCM_STREAM_MAGIC.equals(ByteBuffer.wrap(headerBytes, 0, 4)), + "Invalid GCM stream: magic does not match AGS1"); + + int plainBlockSize = ByteBuffer.wrap(headerBytes, 4, 4).order(ByteOrder.LITTLE_ENDIAN).getInt(); + Preconditions.checkState( + plainBlockSize == Ciphers.PLAIN_BLOCK_SIZE, + "Invalid GCM stream: block size %d != %d", + plainBlockSize, + Ciphers.PLAIN_BLOCK_SIZE); } @Override - public int available() throws IOException { + public int available() { long maxAvailable = plainStreamSize - plainStreamPosition; // See InputStream.available contract if (maxAvailable >= Integer.MAX_VALUE) { @@ -115,9 +91,17 @@ public int available() throws IOException { } } + private int availableInCurrentBlock() { + if (currentPlainBlockIndex < 0) { + return 0; + } + + return currentPlainBlockSize - offsetInBlock(plainStreamPosition); + } + @Override public int read(byte[] b, int off, int len) throws IOException { - Preconditions.checkState(len >= 0, "Negative read length " + len); + Preconditions.checkArgument(len >= 0, "Invalid read length: " + len); if (available() <= 0 && len > 0) { throw new EOFException(); @@ -127,50 +111,51 @@ public int read(byte[] b, int off, int len) throws IOException { return 0; } - boolean isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks); + int totalBytesRead = 0; int resultBufferOffset = off; int remainingBytesToRead = len; while (remainingBytesToRead > 0) { - byte[] plainBlock = decryptNextBlock(isLastBlockInStream); - - int remainingBytesInBlock = plainBlock.length - currentOffsetInPlainBlock; - boolean finishTheBlock = remainingBytesToRead >= remainingBytesInBlock; - int bytesToCopy = finishTheBlock ? remainingBytesInBlock : remainingBytesToRead; - System.arraycopy(plainBlock, currentOffsetInPlainBlock, b, resultBufferOffset, bytesToCopy); - remainingBytesToRead -= bytesToCopy; - resultBufferOffset += bytesToCopy; - currentOffsetInPlainBlock += bytesToCopy; - - boolean endOfStream = isLastBlockInStream && finishTheBlock; - - if (endOfStream) { + int availableInBlock = availableInCurrentBlock(); + if (availableInBlock > 0) { + int bytesToCopy = Math.min(availableInBlock, remainingBytesToRead); + int offsetInBlock = offsetInBlock(plainStreamPosition); + System.arraycopy(currentPlainBlock, offsetInBlock, b, resultBufferOffset, bytesToCopy); + totalBytesRead += bytesToCopy; + remainingBytesToRead -= bytesToCopy; + resultBufferOffset += bytesToCopy; + this.plainStreamPosition += bytesToCopy; + if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) { + // invalidate the current block + this.currentPlainBlockIndex = -1; + } + + } else if (available() > 0) { + decryptBlock(blockIndex(plainStreamPosition)); + + } else { break; } - - if (finishTheBlock) { - currentBlockIndex++; - currentOffsetInPlainBlock = 0; - isLastBlockInStream = (currentBlockIndex + 1 == numberOfBlocks); - } } - plainStreamPosition += len - remainingBytesToRead; - return len - remainingBytesToRead; + // return -1 for EOF + return totalBytesRead > 0 ? totalBytesRead : -1; } @Override public void seek(long newPos) throws IOException { if (newPos < 0) { - throw new IOException("Negative new position " + newPos); + throw new IOException("Invalid position: " + newPos); } else if (newPos > plainStreamSize) { throw new EOFException( - "New position " + newPos + " exceeds the max stream size " + plainStreamSize); + "Invalid position: " + newPos + " > stream length, " + plainStreamSize); } - currentBlockIndex = Math.toIntExact(newPos / plainBlockSize); - currentOffsetInPlainBlock = Math.toIntExact(newPos % plainBlockSize); - plainStreamPosition = newPos; + this.plainStreamPosition = newPos; + if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) { + // invalidate the current block + this.currentPlainBlockIndex = -1; + } } @Override @@ -179,27 +164,19 @@ public long skip(long n) { return 0; } - if (plainStreamPosition == plainStreamSize) { - return 0; + long bytesLeftInStream = plainStreamSize - plainStreamPosition; + if (n > bytesLeftInStream) { + // skip the rest of the stream + this.plainStreamPosition = plainStreamSize; + return bytesLeftInStream; } - long newPosition = plainStreamPosition + n; - - if (newPosition > plainStreamSize) { - long skipped = plainStreamSize - plainStreamPosition; - try { - seek(plainStreamSize); - } catch (IOException e) { - throw new UncheckedIOException(e); - } - return skipped; + this.plainStreamPosition += n; + if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) { + // invalidate the current block + this.currentPlainBlockIndex = -1; } - try { - seek(newPosition); - } catch (IOException e) { - throw new UncheckedIOException(e); - } return n; } @@ -216,59 +193,62 @@ public int read() throws IOException { @Override public void close() throws IOException { sourceStream.close(); - currentDecryptedBlock = null; + this.currentPlainBlock = null; } - static long calculatePlaintextLength(long sourceLength, int plainBlockSize) { - long netSourceFileLength = netSourceFileLength(sourceLength); - - if (netSourceFileLength == 0) { - return 0; + private void decryptBlock(long blockIndex) throws IOException { + if (blockIndex == currentPlainBlockIndex) { + return; } - int cipherBlockSize = plainBlockSize + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH; - int numberOfFullBlocks = Math.toIntExact(netSourceFileLength / cipherBlockSize); - int cipherBytesInLastBlock = - Math.toIntExact(netSourceFileLength - numberOfFullBlocks * cipherBlockSize); - boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); - int plainBytesInLastBlock = - fullBlocksOnly - ? 0 - : (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH); - - return (long) numberOfFullBlocks * plainBlockSize + plainBytesInLastBlock; - } - - private byte[] decryptNextBlock(boolean isLastBlockInStream) throws IOException { - if (currentBlockIndex == currentDecryptedBlockIndex) { - return currentDecryptedBlock; - } - - long blockPositionInStream = blockOffset(currentBlockIndex); + long blockPositionInStream = blockOffset(blockIndex); if (sourceStream.getPos() != blockPositionInStream) { + if (sourceStream.getPos() == 0) { + validateHeader(); + } + sourceStream.seek(blockPositionInStream); } - int currentCipherBlockSize = isLastBlockInStream ? lastCipherBlockSize : cipherBlockSize; - IOUtil.readFully(sourceStream, cipherBlockBuffer, 0, currentCipherBlockSize); + boolean isLastBlock = blockIndex == numBlocks - 1; + int cipherBlockSize = isLastBlock ? lastCipherBlockSize : Ciphers.CIPHER_BLOCK_SIZE; + IOUtil.readFully(sourceStream, cipherBlockBuffer, 0, cipherBlockSize); - byte[] aad = Ciphers.streamBlockAAD(fileAADPrefix, currentBlockIndex); - byte[] result = gcmDecryptor.decrypt(cipherBlockBuffer, 0, currentCipherBlockSize, aad); - currentDecryptedBlockIndex = currentBlockIndex; - currentDecryptedBlock = result; - return result; + // TODO: the AAD should probably use a long block index. + byte[] blockAAD = Ciphers.streamBlockAAD(fileAADPrefix, Math.toIntExact(blockIndex)); + this.currentPlainBlock = decryptor.decrypt(cipherBlockBuffer, 0, cipherBlockSize, blockAAD); + this.currentPlainBlockSize = cipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH; + this.currentPlainBlockIndex = blockIndex; } - private long blockOffset(int blockIndex) { - return (long) blockIndex * cipherBlockSize + Ciphers.GCM_STREAM_HEADER_LENGTH; + private static long blockIndex(long plainPosition) { + return plainPosition / Ciphers.PLAIN_BLOCK_SIZE; } - private static long netSourceFileLength(long sourceFileLength) { - long netSourceLength = sourceFileLength - Ciphers.GCM_STREAM_HEADER_LENGTH; - Preconditions.checkArgument( - netSourceLength >= 0, - "Source length " + sourceFileLength + " is shorter than GCM prefix. File is not encrypted"); + private static int offsetInBlock(long plainPosition) { + return Math.toIntExact(plainPosition % Ciphers.PLAIN_BLOCK_SIZE); + } + + private static long blockOffset(long blockIndex) { + return blockIndex * Ciphers.CIPHER_BLOCK_SIZE + Ciphers.GCM_STREAM_HEADER_LENGTH; + } + + static long calculatePlaintextLength(long sourceLength) { + long streamLength = sourceLength - Ciphers.GCM_STREAM_HEADER_LENGTH; + + if (streamLength == 0) { + return 0; + } + + long numberOfFullBlocks = streamLength / Ciphers.CIPHER_BLOCK_SIZE; + long fullBlockSize = numberOfFullBlocks * Ciphers.CIPHER_BLOCK_SIZE; + long cipherBytesInLastBlock = streamLength - fullBlockSize; + boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); + long plainBytesInLastBlock = + fullBlocksOnly + ? 0 + : (cipherBytesInLastBlock - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH); - return netSourceLength; + return (numberOfFullBlocks * Ciphers.PLAIN_BLOCK_SIZE) + plainBytesInLastBlock; } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index ce165c39c70f..2739302b9f4e 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -24,7 +24,6 @@ import org.apache.iceberg.io.PositionOutputStream; public class AesGcmOutputStream extends PositionOutputStream { - public static final int plainBlockSize = 1024 * 1024; private final Ciphers.AesGcmEncryptor gcmEncryptor; private final PositionOutputStream targetStream; @@ -39,7 +38,7 @@ public class AesGcmOutputStream extends PositionOutputStream { throws IOException { this.targetStream = targetStream; this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey); - this.plainBlockBuffer = new byte[plainBlockSize]; + this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE]; this.positionInBuffer = 0; this.streamPosition = 0; this.currentBlockIndex = 0; @@ -49,7 +48,7 @@ public class AesGcmOutputStream extends PositionOutputStream { ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH) .order(ByteOrder.LITTLE_ENDIAN) .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) - .putInt(plainBlockSize) + .putInt(Ciphers.PLAIN_BLOCK_SIZE) .array(); targetStream.write(headerBytes); } @@ -69,12 +68,12 @@ public void write(byte[] b, int off, int len) throws IOException { int offset = off; while (remaining > 0) { - int freeBlockBytes = plainBlockSize - positionInBuffer; + int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer; int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining; System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite); positionInBuffer += toWrite; - if (positionInBuffer == plainBlockSize) { + if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) { encryptAndWriteBlock(); positionInBuffer = 0; } diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index cf43880a7475..d68d0774ff9c 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -30,12 +30,16 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class Ciphers { + public static final int PLAIN_BLOCK_SIZE = 1024 * 1024; public static final int NONCE_LENGTH = 12; public static final int GCM_TAG_LENGTH = 16; + public static final int CIPHER_BLOCK_SIZE = + PLAIN_BLOCK_SIZE + NONCE_LENGTH + GCM_TAG_LENGTH; public static final String GCM_STREAM_MAGIC_STRING = "AGS1"; static final byte[] GCM_STREAM_MAGIC_ARRAY = GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8); + static final ByteBuffer GCM_STREAM_MAGIC = ByteBuffer.wrap(GCM_STREAM_MAGIC_ARRAY).asReadOnlyBuffer(); static final int GCM_STREAM_HEADER_LENGTH = GCM_STREAM_MAGIC_ARRAY.length + 4; // magic_len + block_size_len diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java index d6905f1306f5..ec929efe2d7e 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -57,9 +57,9 @@ public void testEmptyFile() throws IOException { @Test public void testRandomWriteRead() throws IOException { Random random = new Random(); - int smallerThanBlock = (int) (AesGcmOutputStream.plainBlockSize * 0.5); - int largerThanBlock = (int) (AesGcmOutputStream.plainBlockSize * 1.5); - int alignedWithBlock = AesGcmOutputStream.plainBlockSize; + int smallerThanBlock = (int) (Ciphers.PLAIN_BLOCK_SIZE * 0.5); + int largerThanBlock = (int) (Ciphers.PLAIN_BLOCK_SIZE * 1.5); + int alignedWithBlock = Ciphers.PLAIN_BLOCK_SIZE; int[] testFileSizes = { smallerThanBlock, largerThanBlock, @@ -162,9 +162,9 @@ public void testRandomWriteRead() throws IOException { public void testAlignedWriteRead() throws IOException { Random random = new Random(); int[] testFileSizes = { - AesGcmOutputStream.plainBlockSize, - AesGcmOutputStream.plainBlockSize + 1, - AesGcmOutputStream.plainBlockSize - 1 + Ciphers.PLAIN_BLOCK_SIZE, + Ciphers.PLAIN_BLOCK_SIZE + 1, + Ciphers.PLAIN_BLOCK_SIZE - 1 }; for (int testFileSize : testFileSizes) { @@ -181,7 +181,7 @@ public void testAlignedWriteRead() throws IOException { PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); int offset = 0; - int chunkLen = AesGcmOutputStream.plainBlockSize; + int chunkLen = Ciphers.PLAIN_BLOCK_SIZE; int left = testFileSize; while (left > 0) { @@ -204,7 +204,7 @@ public void testAlignedWriteRead() throws IOException { Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); offset = 0; - chunkLen = AesGcmOutputStream.plainBlockSize; + chunkLen = Ciphers.PLAIN_BLOCK_SIZE; byte[] chunk = new byte[chunkLen]; left = testFileSize; From 82fc5b0f961ecd26b372aa8fd5a911e36ef8748c Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Mon, 31 Jul 2023 10:37:12 +0300 Subject: [PATCH 10/16] spotless apply --- .../org/apache/iceberg/encryption/AesGcmInputStream.java | 9 ++++++--- .../main/java/org/apache/iceberg/encryption/Ciphers.java | 6 +++--- .../org/apache/iceberg/encryption/TestGcmStreams.java | 4 +--- 3 files changed, 10 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index 0bbb8ad4556e..b2bbfeaa4cc6 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -58,10 +58,13 @@ public class AesGcmInputStream extends SeekableInputStream { int cipherBytesInLastBlock = Math.toIntExact(streamLength - cipherFullBlockLength); boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); this.numBlocks = fullBlocksOnly ? numFullBlocks : numFullBlocks + 1; - this.lastCipherBlockSize = fullBlocksOnly ? Ciphers.CIPHER_BLOCK_SIZE : cipherBytesInLastBlock; // never 0 + this.lastCipherBlockSize = + fullBlocksOnly ? Ciphers.CIPHER_BLOCK_SIZE : cipherBytesInLastBlock; // never 0 - long lastPlainBlockSize = lastCipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH; - this.plainStreamSize = numFullBlocks * Ciphers.PLAIN_BLOCK_SIZE + (fullBlocksOnly ? 0 : lastPlainBlockSize); + long lastPlainBlockSize = + (long) lastCipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH; + this.plainStreamSize = + numFullBlocks * Ciphers.PLAIN_BLOCK_SIZE + (fullBlocksOnly ? 0 : lastPlainBlockSize); } private void validateHeader() throws IOException { diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index d68d0774ff9c..3cb3e900a406 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -33,13 +33,13 @@ public class Ciphers { public static final int PLAIN_BLOCK_SIZE = 1024 * 1024; public static final int NONCE_LENGTH = 12; public static final int GCM_TAG_LENGTH = 16; - public static final int CIPHER_BLOCK_SIZE = - PLAIN_BLOCK_SIZE + NONCE_LENGTH + GCM_TAG_LENGTH; + public static final int CIPHER_BLOCK_SIZE = PLAIN_BLOCK_SIZE + NONCE_LENGTH + GCM_TAG_LENGTH; public static final String GCM_STREAM_MAGIC_STRING = "AGS1"; static final byte[] GCM_STREAM_MAGIC_ARRAY = GCM_STREAM_MAGIC_STRING.getBytes(StandardCharsets.UTF_8); - static final ByteBuffer GCM_STREAM_MAGIC = ByteBuffer.wrap(GCM_STREAM_MAGIC_ARRAY).asReadOnlyBuffer(); + static final ByteBuffer GCM_STREAM_MAGIC = + ByteBuffer.wrap(GCM_STREAM_MAGIC_ARRAY).asReadOnlyBuffer(); static final int GCM_STREAM_HEADER_LENGTH = GCM_STREAM_MAGIC_ARRAY.length + 4; // magic_len + block_size_len diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java index ec929efe2d7e..0b3f085bc9d7 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -162,9 +162,7 @@ public void testRandomWriteRead() throws IOException { public void testAlignedWriteRead() throws IOException { Random random = new Random(); int[] testFileSizes = { - Ciphers.PLAIN_BLOCK_SIZE, - Ciphers.PLAIN_BLOCK_SIZE + 1, - Ciphers.PLAIN_BLOCK_SIZE - 1 + Ciphers.PLAIN_BLOCK_SIZE, Ciphers.PLAIN_BLOCK_SIZE + 1, Ciphers.PLAIN_BLOCK_SIZE - 1 }; for (int testFileSize : testFileSizes) { From 375ef2012c312a26dcec852f7bebd9ac03fea88b Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Mon, 31 Jul 2023 15:31:23 +0300 Subject: [PATCH 11/16] address review comments --- .../iceberg/encryption/AesGcmInputStream.java | 9 ++++++- .../apache/iceberg/encryption/Ciphers.java | 27 ++++++++++++------- .../iceberg/encryption/TestCiphers.java | 16 +++++++++++ 3 files changed, 42 insertions(+), 10 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index b2bbfeaa4cc6..a63134f31d9d 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -39,6 +39,7 @@ public class AesGcmInputStream extends SeekableInputStream { private long currentPlainBlockIndex; private byte[] currentPlainBlock; private int currentPlainBlockSize; + private byte[] singleByte; AesGcmInputStream( SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix) { @@ -65,6 +66,7 @@ public class AesGcmInputStream extends SeekableInputStream { (long) lastCipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH; this.plainStreamSize = numFullBlocks * Ciphers.PLAIN_BLOCK_SIZE + (fullBlocksOnly ? 0 : lastPlainBlockSize); + this.singleByte = new byte[1]; } private void validateHeader() throws IOException { @@ -190,7 +192,12 @@ public long getPos() throws IOException { @Override public int read() throws IOException { - throw new UnsupportedOperationException(); + int read = read(singleByte); + if (read == -1) { + return -1; + } + + return singleByte[0]; } @Override diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index 3cb3e900a406..443bd5e87dab 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -76,6 +76,7 @@ public byte[] encrypt(byte[] plaintext, byte[] aad) { } public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) { + Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength); byte[] nonce = new byte[NONCE_LENGTH]; randomGenerator.nextBytes(nonce); int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH; @@ -87,7 +88,17 @@ public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength if (null != aad) { cipher.updateAAD(aad); } - cipher.doFinal(plaintext, plaintextOffset, plaintextLength, cipherText, NONCE_LENGTH); + int enciphered = + cipher.doFinal(plaintext, plaintextOffset, plaintextLength, cipherText, NONCE_LENGTH); + + if (enciphered != plaintextLength + GCM_TAG_LENGTH) { + throw new RuntimeException( + "Wrong number of enciphered bytes: " + + enciphered + + ". Must be " + + plaintextLength + + GCM_TAG_LENGTH); + } } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to encrypt", e); } @@ -133,22 +144,20 @@ public byte[] decrypt( + " 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, ciphertextOffset, nonce, 0, NONCE_LENGTH); - - int inputLength = ciphertextLength - NONCE_LENGTH; try { - GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce); + GCMParameterSpec spec = + new GCMParameterSpec(GCM_TAG_LENGTH_BITS, ciphertext, ciphertextOffset, NONCE_LENGTH); cipher.init(Cipher.DECRYPT_MODE, aesKey, spec); if (null != aad) { cipher.updateAAD(aad); } - return cipher.doFinal(ciphertext, ciphertextOffset + NONCE_LENGTH, inputLength); + // For java Cipher, the nonce is not part of ciphertext + return cipher.doFinal( + ciphertext, ciphertextOffset + NONCE_LENGTH, ciphertextLength - NONCE_LENGTH); } catch (AEADBadTagException e) { throw new RuntimeException( "GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" - + "data. AES GCM doesn't differentiate between these two.", + + " data. AES GCM doesn't differentiate between these two.", e); } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to decrypt", e); diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java index a428586c3b46..7bc7625c1b4c 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java @@ -22,6 +22,7 @@ import java.nio.charset.StandardCharsets; import java.security.SecureRandom; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; public class TestCiphers { @@ -51,6 +52,21 @@ private void testEncryptDecrypt(byte[] aad) { Ciphers.AesGcmDecryptor decryptor = new Ciphers.AesGcmDecryptor(key); byte[] decryptedText = decryptor.decrypt(ciphertext, aad); assertThat(decryptedText).as("Key length " + keyLength).isEqualTo(plaintext); + + // Test bad aad + final byte[] badAad = (aad == null) ? new byte[1] : aad; + badAad[0]++; + + Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, badAad)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("GCM tag check failed"); + + // Test content corruption + ciphertext[ciphertext.length / 2]++; + + Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, aad)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("GCM tag check failed"); } } } From cf8827f42263d08dab75c3d4d0549d7e0c646620 Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Wed, 2 Aug 2023 10:54:01 +0300 Subject: [PATCH 12/16] address new review comments --- .../iceberg/encryption/AesGcmInputStream.java | 30 +++----- .../iceberg/encryption/AesGcmOutputFile.java | 17 +---- .../encryption/AesGcmOutputStream.java | 75 ++++++++++++------- .../apache/iceberg/encryption/Ciphers.java | 57 +++++++++++--- .../iceberg/encryption/TestCiphers.java | 1 + 5 files changed, 110 insertions(+), 70 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index a63134f31d9d..e033dd62ee14 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -30,16 +30,16 @@ public class AesGcmInputStream extends SeekableInputStream { private final SeekableInputStream sourceStream; private final byte[] fileAADPrefix; private final Ciphers.AesGcmDecryptor decryptor; - private final byte[] cipherBlockBuffer; private final long numBlocks; private final int lastCipherBlockSize; private final long plainStreamSize; + private final byte[] singleByte; private long plainStreamPosition; private long currentPlainBlockIndex; + private byte[] cipherBlockBuffer; private byte[] currentPlainBlock; private int currentPlainBlockSize; - private byte[] singleByte; AesGcmInputStream( SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix) { @@ -47,10 +47,9 @@ public class AesGcmInputStream extends SeekableInputStream { this.fileAADPrefix = fileAADPrefix; this.decryptor = new Ciphers.AesGcmDecryptor(aesKey); this.cipherBlockBuffer = new byte[Ciphers.CIPHER_BLOCK_SIZE]; - + this.currentPlainBlock = new byte[Ciphers.PLAIN_BLOCK_SIZE]; this.plainStreamPosition = 0; this.currentPlainBlockIndex = -1; - this.currentPlainBlock = null; this.currentPlainBlockSize = 0; long streamLength = sourceLength - Ciphers.GCM_STREAM_HEADER_LENGTH; @@ -97,7 +96,7 @@ public int available() { } private int availableInCurrentBlock() { - if (currentPlainBlockIndex < 0) { + if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) { return 0; } @@ -130,11 +129,6 @@ public int read(byte[] b, int off, int len) throws IOException { remainingBytesToRead -= bytesToCopy; resultBufferOffset += bytesToCopy; this.plainStreamPosition += bytesToCopy; - if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) { - // invalidate the current block - this.currentPlainBlockIndex = -1; - } - } else if (available() > 0) { decryptBlock(blockIndex(plainStreamPosition)); @@ -157,10 +151,6 @@ public void seek(long newPos) throws IOException { } this.plainStreamPosition = newPos; - if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) { - // invalidate the current block - this.currentPlainBlockIndex = -1; - } } @Override @@ -177,10 +167,6 @@ public long skip(long n) { } this.plainStreamPosition += n; - if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) { - // invalidate the current block - this.currentPlainBlockIndex = -1; - } return n; } @@ -197,13 +183,16 @@ public int read() throws IOException { return -1; } - return singleByte[0]; + int unsignedByte = singleByte[0] >= 0 ? singleByte[0] : 256 + singleByte[0]; + + return unsignedByte; } @Override public void close() throws IOException { sourceStream.close(); this.currentPlainBlock = null; + this.cipherBlockBuffer = null; } private void decryptBlock(long blockIndex) throws IOException { @@ -224,9 +213,8 @@ private void decryptBlock(long blockIndex) throws IOException { int cipherBlockSize = isLastBlock ? lastCipherBlockSize : Ciphers.CIPHER_BLOCK_SIZE; IOUtil.readFully(sourceStream, cipherBlockBuffer, 0, cipherBlockSize); - // TODO: the AAD should probably use a long block index. byte[] blockAAD = Ciphers.streamBlockAAD(fileAADPrefix, Math.toIntExact(blockIndex)); - this.currentPlainBlock = decryptor.decrypt(cipherBlockBuffer, 0, cipherBlockSize, blockAAD); + decryptor.decrypt(cipherBlockBuffer, 0, cipherBlockSize, currentPlainBlock, 0, blockAAD); this.currentPlainBlockSize = cipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH; this.currentPlainBlockIndex = blockIndex; } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java index 15d3c19e56c3..c3f5ee2d0327 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java @@ -18,8 +18,6 @@ */ package org.apache.iceberg.encryption; -import java.io.IOException; -import java.io.UncheckedIOException; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.io.PositionOutputStream; @@ -37,21 +35,12 @@ public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey, byte[] fileAADPre @Override public PositionOutputStream create() { - try { - return new AesGcmOutputStream(targetFile.create(), dataKey, fileAADPrefix); - } catch (IOException e) { - throw new UncheckedIOException("Failed to create GCM stream for " + targetFile.location(), e); - } + return new AesGcmOutputStream(targetFile.create(), dataKey, fileAADPrefix); } @Override public PositionOutputStream createOrOverwrite() { - try { - return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, fileAADPrefix); - } catch (IOException e) { - throw new UncheckedIOException( - "Failed to create or overwrite GCM stream for " + targetFile.location(), e); - } + return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, fileAADPrefix); } @Override @@ -61,6 +50,6 @@ public String location() { @Override public InputFile toInputFile() { - throw new UnsupportedOperationException(); + return new AesGcmInputFile(targetFile.toInputFile(), dataKey, fileAADPrefix); } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index 2739302b9f4e..aa1ba097bb37 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -27,58 +27,61 @@ public class AesGcmOutputStream extends PositionOutputStream { private final Ciphers.AesGcmEncryptor gcmEncryptor; private final PositionOutputStream targetStream; - private final byte[] plainBlockBuffer; private final byte[] fileAadPrefix; + private final byte[] singleByte; - private int positionInBuffer; + private byte[] plainBlock; + private byte[] cipherBlock; + private int positionInPlainBlock; private long streamPosition; private int currentBlockIndex; + private boolean isHeaderWritten; - AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) - throws IOException { + AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) { this.targetStream = targetStream; this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey); - this.plainBlockBuffer = new byte[Ciphers.PLAIN_BLOCK_SIZE]; - this.positionInBuffer = 0; + this.plainBlock = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + this.cipherBlock = new byte[Ciphers.CIPHER_BLOCK_SIZE]; + this.positionInPlainBlock = 0; this.streamPosition = 0; this.currentBlockIndex = 0; this.fileAadPrefix = fileAadPrefix; - - byte[] headerBytes = - ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH) - .order(ByteOrder.LITTLE_ENDIAN) - .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) - .putInt(Ciphers.PLAIN_BLOCK_SIZE) - .array(); - targetStream.write(headerBytes); + this.isHeaderWritten = false; + this.singleByte = new byte[1]; } @Override public void write(int b) throws IOException { - throw new UnsupportedOperationException(); + singleByte[0] = (byte) (b & 0x000000FF); + write(singleByte); } @Override public void write(byte[] b, int off, int len) throws IOException { + if (!isHeaderWritten) { + writeHeader(); + } + if (b.length - off < len) { throw new IOException( "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len); } + int remaining = len; int offset = off; while (remaining > 0) { - int freeBlockBytes = Ciphers.PLAIN_BLOCK_SIZE - positionInBuffer; - int toWrite = freeBlockBytes <= remaining ? freeBlockBytes : remaining; + int freeBlockBytes = plainBlock.length - positionInPlainBlock; + int toWrite = Math.min(freeBlockBytes, remaining); - System.arraycopy(b, offset, plainBlockBuffer, positionInBuffer, toWrite); - positionInBuffer += toWrite; - if (positionInBuffer == Ciphers.PLAIN_BLOCK_SIZE) { - encryptAndWriteBlock(); - positionInBuffer = 0; - } + System.arraycopy(b, offset, plainBlock, positionInPlainBlock, toWrite); + positionInPlainBlock += toWrite; offset += toWrite; remaining -= toWrite; + + if (positionInPlainBlock == Ciphers.PLAIN_BLOCK_SIZE) { + encryptAndWriteBlock(); + } } streamPosition += len; @@ -96,10 +99,28 @@ public void flush() throws IOException { @Override public void close() throws IOException { - if (positionInBuffer > 0) { + if (!isHeaderWritten) { + writeHeader(); + } + + if (positionInPlainBlock > 0) { encryptAndWriteBlock(); } + targetStream.close(); + plainBlock = null; + cipherBlock = null; + } + + private void writeHeader() throws IOException { + byte[] headerBytes = + ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH) + .order(ByteOrder.LITTLE_ENDIAN) + .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) + .putInt(Ciphers.PLAIN_BLOCK_SIZE) + .array(); + targetStream.write(headerBytes); + isHeaderWritten = true; } private void encryptAndWriteBlock() throws IOException { @@ -108,8 +129,10 @@ private void encryptAndWriteBlock() throws IOException { } byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex); - byte[] cipherBlockBuffer = gcmEncryptor.encrypt(plainBlockBuffer, 0, positionInBuffer, aad); + int ciphertextLength = + gcmEncryptor.encrypt(plainBlock, 0, positionInPlainBlock, cipherBlock, 0, aad); + targetStream.write(cipherBlock, 0, ciphertextLength); + positionInPlainBlock = 0; currentBlockIndex++; - targetStream.write(cipherBlockBuffer); } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index 443bd5e87dab..24ed27017247 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -51,6 +51,7 @@ public static class AesGcmEncryptor { private final SecretKeySpec aesKey; private final Cipher cipher; private final SecureRandom randomGenerator; + private final byte[] nonce; public AesGcmEncryptor(byte[] keyBytes) { Preconditions.checkArgument(keyBytes != null, "Key can't be null"); @@ -69,6 +70,7 @@ public AesGcmEncryptor(byte[] keyBytes) { } this.randomGenerator = new SecureRandom(); + this.nonce = new byte[NONCE_LENGTH]; } public byte[] encrypt(byte[] plaintext, byte[] aad) { @@ -76,11 +78,22 @@ public byte[] encrypt(byte[] plaintext, byte[] aad) { } public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) { - Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength); - byte[] nonce = new byte[NONCE_LENGTH]; - randomGenerator.nextBytes(nonce); int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH; byte[] cipherText = new byte[cipherTextLength]; + encrypt(plaintext, plaintextOffset, plaintextLength, cipherText, 0, aad); + return cipherText; + } + + public int encrypt( + byte[] plaintext, + int plaintextOffset, + int plaintextLength, + byte[] ciphertextBuffer, + int ciphertextOffset, + byte[] aad) { + Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength); + randomGenerator.nextBytes(nonce); + int enciphered; try { GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce); @@ -88,8 +101,13 @@ public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength if (null != aad) { cipher.updateAAD(aad); } - int enciphered = - cipher.doFinal(plaintext, plaintextOffset, plaintextLength, cipherText, NONCE_LENGTH); + enciphered = + cipher.doFinal( + plaintext, + plaintextOffset, + plaintextLength, + ciphertextBuffer, + ciphertextOffset + NONCE_LENGTH); if (enciphered != plaintextLength + GCM_TAG_LENGTH) { throw new RuntimeException( @@ -104,9 +122,9 @@ public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength } // Add the nonce - System.arraycopy(nonce, 0, cipherText, 0, NONCE_LENGTH); + System.arraycopy(nonce, 0, ciphertextBuffer, ciphertextOffset, NONCE_LENGTH); - return cipherText; + return enciphered + NONCE_LENGTH; } } @@ -137,12 +155,26 @@ public byte[] decrypt(byte[] ciphertext, byte[] aad) { public byte[] decrypt( byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) { + int plaintextLength = ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH; + byte[] plaintext = new byte[plaintextLength]; + decrypt(ciphertext, ciphertextOffset, ciphertextLength, plaintext, 0, aad); + return plaintext; + } + + public int decrypt( + byte[] ciphertext, + int ciphertextOffset, + int ciphertextLength, + byte[] plaintextBuffer, + int plaintextOffset, + byte[] aad) { Preconditions.checkState( ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1, "Cannot decrypt cipher text of length " + ciphertext.length + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" + " with AES GCM cipher"); + int plaintextLength; try { GCMParameterSpec spec = @@ -152,8 +184,13 @@ public byte[] decrypt( cipher.updateAAD(aad); } // For java Cipher, the nonce is not part of ciphertext - return cipher.doFinal( - ciphertext, ciphertextOffset + NONCE_LENGTH, ciphertextLength - NONCE_LENGTH); + plaintextLength = + cipher.doFinal( + ciphertext, + ciphertextOffset + NONCE_LENGTH, + ciphertextLength - NONCE_LENGTH, + plaintextBuffer, + plaintextOffset); } catch (AEADBadTagException e) { throw new RuntimeException( "GCM tag check failed. Possible reasons: wrong decryption key; or corrupt/tampered" @@ -162,6 +199,8 @@ public byte[] decrypt( } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to decrypt", e); } + + return plaintextLength; } } diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java index 7bc7625c1b4c..40264ca187b2 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java @@ -41,6 +41,7 @@ public void testAAD() { private void testEncryptDecrypt(byte[] aad) { SecureRandom random = new SecureRandom(); int[] aesKeyLengthArray = {16, 24, 32}; + for (int keyLength : aesKeyLengthArray) { byte[] key = new byte[keyLength]; random.nextBytes(key); From de84a93592c755b33c721379b27f026af287f12e Mon Sep 17 00:00:00 2001 From: Gidon Gershinsky Date: Mon, 14 Aug 2023 14:09:30 +0300 Subject: [PATCH 13/16] address comments --- .../encryption/AesGcmOutputStream.java | 21 +++++--- .../apache/iceberg/encryption/Ciphers.java | 13 +++-- .../iceberg/encryption/TestCiphers.java | 50 +++++++++++++------ 3 files changed, 55 insertions(+), 29 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index aa1ba097bb37..3a3fbd6b3f3e 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -25,6 +25,12 @@ public class AesGcmOutputStream extends PositionOutputStream { + private static final byte[] HEADER_BYTES = + ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH) + .order(ByteOrder.LITTLE_ENDIAN) + .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) + .putInt(Ciphers.PLAIN_BLOCK_SIZE) + .array(); private final Ciphers.AesGcmEncryptor gcmEncryptor; private final PositionOutputStream targetStream; private final byte[] fileAadPrefix; @@ -113,19 +119,18 @@ public void close() throws IOException { } private void writeHeader() throws IOException { - byte[] headerBytes = - ByteBuffer.allocate(Ciphers.GCM_STREAM_HEADER_LENGTH) - .order(ByteOrder.LITTLE_ENDIAN) - .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) - .putInt(Ciphers.PLAIN_BLOCK_SIZE) - .array(); - targetStream.write(headerBytes); + + targetStream.write(HEADER_BYTES); isHeaderWritten = true; } private void encryptAndWriteBlock() throws IOException { if (currentBlockIndex == Integer.MAX_VALUE) { - throw new IOException("Too many blocks - exceed Integer.MAX_VALUE"); + throw new IOException("Cannot write block: exceeded Integer.MAX_VALUE blocks"); + } + + if (positionInPlainBlock == 0) { + throw new IOException("Empty plain block"); } byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex); diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index 24ed27017247..381e1372952b 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -91,7 +91,8 @@ public int encrypt( byte[] ciphertextBuffer, int ciphertextOffset, byte[] aad) { - Preconditions.checkArgument(plaintextLength > 0, "Wrong plaintextLength " + plaintextLength); + Preconditions.checkArgument( + plaintextLength > 0, "Invalid plain text length: %s", plaintextLength); randomGenerator.nextBytes(nonce); int enciphered; @@ -101,6 +102,8 @@ public int encrypt( if (null != aad) { cipher.updateAAD(aad); } + + // doFinal encrypts and adds a GCM tag. The nonce is added later. enciphered = cipher.doFinal( plaintext, @@ -111,11 +114,11 @@ public int encrypt( if (enciphered != plaintextLength + GCM_TAG_LENGTH) { throw new RuntimeException( - "Wrong number of enciphered bytes: " - + enciphered - + ". Must be " + "Failed to encrypt block: expected " + plaintextLength - + GCM_TAG_LENGTH); + + GCM_TAG_LENGTH + + " encrypted bytes but produced bytes " + + enciphered); } } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to encrypt", e); diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java index 40264ca187b2..eedcea4ea2c1 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java @@ -29,16 +29,29 @@ public class TestCiphers { @Test public void testBasicEncrypt() { - testEncryptDecrypt(null); + testEncryptDecrypt(null, true, false, false); } @Test public void testAAD() { byte[] aad = "abcd".getBytes(StandardCharsets.UTF_8); - testEncryptDecrypt(aad); + testEncryptDecrypt(aad, true, false, false); } - private void testEncryptDecrypt(byte[] aad) { + @Test + public void testBadAAD() { + byte[] aad = "abcd".getBytes(StandardCharsets.UTF_8); + testEncryptDecrypt(aad, false, true, false); + } + + @Test + public void testContentCorruption() { + byte[] aad = "abcd".getBytes(StandardCharsets.UTF_8); + testEncryptDecrypt(aad, false, false, true); + } + + private void testEncryptDecrypt( + byte[] aad, boolean testDecrypt, boolean testBadAad, boolean testCorruption) { SecureRandom random = new SecureRandom(); int[] aesKeyLengthArray = {16, 24, 32}; @@ -51,23 +64,28 @@ private void testEncryptDecrypt(byte[] aad) { byte[] ciphertext = encryptor.encrypt(plaintext, aad); Ciphers.AesGcmDecryptor decryptor = new Ciphers.AesGcmDecryptor(key); - byte[] decryptedText = decryptor.decrypt(ciphertext, aad); - assertThat(decryptedText).as("Key length " + keyLength).isEqualTo(plaintext); - // Test bad aad - final byte[] badAad = (aad == null) ? new byte[1] : aad; - badAad[0]++; + if (testDecrypt) { + byte[] decryptedText = decryptor.decrypt(ciphertext, aad); + assertThat(decryptedText).as("Key length " + keyLength).isEqualTo(plaintext); + } + + if (testBadAad) { + final byte[] badAad = (aad == null) ? new byte[1] : aad; + badAad[0]++; - Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, badAad)) - .isInstanceOf(RuntimeException.class) - .hasMessageContaining("GCM tag check failed"); + Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, badAad)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("GCM tag check failed"); + } - // Test content corruption - ciphertext[ciphertext.length / 2]++; + if (testCorruption) { + ciphertext[ciphertext.length / 2]++; - Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, aad)) - .isInstanceOf(RuntimeException.class) - .hasMessageContaining("GCM tag check failed"); + Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, aad)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("GCM tag check failed"); + } } } } From a64f4f4bcba01d9e703905cf840eea40a404e024 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 4 Sep 2023 12:35:29 -0700 Subject: [PATCH 14/16] Add tests and minor updates to streams. --- .../iceberg/encryption/AesGcmInputStream.java | 10 +- .../encryption/AesGcmOutputStream.java | 33 ++-- .../apache/iceberg/encryption/Ciphers.java | 50 +++--- .../iceberg/encryption/TestGcmStreams.java | 166 ++++++++++++++++++ 4 files changed, 210 insertions(+), 49 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index e033dd62ee14..138dc2dd2a89 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -30,6 +30,8 @@ public class AesGcmInputStream extends SeekableInputStream { private final SeekableInputStream sourceStream; private final byte[] fileAADPrefix; private final Ciphers.AesGcmDecryptor decryptor; + private final byte[] cipherBlockBuffer; + private final byte[] currentPlainBlock; private final long numBlocks; private final int lastCipherBlockSize; private final long plainStreamSize; @@ -37,8 +39,6 @@ public class AesGcmInputStream extends SeekableInputStream { private long plainStreamPosition; private long currentPlainBlockIndex; - private byte[] cipherBlockBuffer; - private byte[] currentPlainBlock; private int currentPlainBlockSize; AesGcmInputStream( @@ -183,16 +183,12 @@ public int read() throws IOException { return -1; } - int unsignedByte = singleByte[0] >= 0 ? singleByte[0] : 256 + singleByte[0]; - - return unsignedByte; + return singleByte[0] >= 0 ? singleByte[0] : 256 + singleByte[0]; } @Override public void close() throws IOException { sourceStream.close(); - this.currentPlainBlock = null; - this.cipherBlockBuffer = null; } private void decryptBlock(long blockIndex) throws IOException { diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index 3a3fbd6b3f3e..ca718603458c 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -22,6 +22,7 @@ import java.nio.ByteBuffer; import java.nio.ByteOrder; import org.apache.iceberg.io.PositionOutputStream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class AesGcmOutputStream extends PositionOutputStream { @@ -31,29 +32,30 @@ public class AesGcmOutputStream extends PositionOutputStream { .put(Ciphers.GCM_STREAM_MAGIC_ARRAY) .putInt(Ciphers.PLAIN_BLOCK_SIZE) .array(); + private final Ciphers.AesGcmEncryptor gcmEncryptor; private final PositionOutputStream targetStream; private final byte[] fileAadPrefix; private final byte[] singleByte; + private final byte[] plainBlock; + private final byte[] cipherBlock; - private byte[] plainBlock; - private byte[] cipherBlock; private int positionInPlainBlock; - private long streamPosition; private int currentBlockIndex; private boolean isHeaderWritten; + private boolean lastBlockWritten; AesGcmOutputStream(PositionOutputStream targetStream, byte[] aesKey, byte[] fileAadPrefix) { this.targetStream = targetStream; this.gcmEncryptor = new Ciphers.AesGcmEncryptor(aesKey); + this.fileAadPrefix = fileAadPrefix; + this.singleByte = new byte[1]; this.plainBlock = new byte[Ciphers.PLAIN_BLOCK_SIZE]; this.cipherBlock = new byte[Ciphers.CIPHER_BLOCK_SIZE]; this.positionInPlainBlock = 0; - this.streamPosition = 0; this.currentBlockIndex = 0; - this.fileAadPrefix = fileAadPrefix; this.isHeaderWritten = false; - this.singleByte = new byte[1]; + this.lastBlockWritten = false; } @Override @@ -85,17 +87,15 @@ public void write(byte[] b, int off, int len) throws IOException { offset += toWrite; remaining -= toWrite; - if (positionInPlainBlock == Ciphers.PLAIN_BLOCK_SIZE) { + if (positionInPlainBlock == plainBlock.length) { encryptAndWriteBlock(); } } - - streamPosition += len; } @Override public long getPos() throws IOException { - return streamPosition; + return (long) currentBlockIndex * Ciphers.PLAIN_BLOCK_SIZE + positionInPlainBlock; } @Override @@ -114,23 +114,28 @@ public void close() throws IOException { } targetStream.close(); - plainBlock = null; - cipherBlock = null; } private void writeHeader() throws IOException { - targetStream.write(HEADER_BYTES); isHeaderWritten = true; } private void encryptAndWriteBlock() throws IOException { + Preconditions.checkState( + !lastBlockWritten, "Cannot encrypt block: a partial block has already been written"); + if (currentBlockIndex == Integer.MAX_VALUE) { throw new IOException("Cannot write block: exceeded Integer.MAX_VALUE blocks"); } if (positionInPlainBlock == 0) { - throw new IOException("Empty plain block"); + return; + } + + if (positionInPlainBlock != plainBlock.length) { + // signal that a partial block has been written and must be the last + this.lastBlockWritten = true; } byte[] aad = Ciphers.streamBlockAAD(fileAadPrefix, currentBlockIndex); diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index 381e1372952b..cf45b99f5c43 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -54,20 +54,8 @@ public static class AesGcmEncryptor { private final byte[] nonce; public AesGcmEncryptor(byte[] keyBytes) { - Preconditions.checkArgument(keyBytes != null, "Key can't be null"); - 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.aesKey = newKey(keyBytes); + this.cipher = newCipher(); this.randomGenerator = new SecureRandom(); this.nonce = new byte[NONCE_LENGTH]; @@ -136,20 +124,8 @@ public static class AesGcmDecryptor { private final Cipher cipher; public AesGcmDecryptor(byte[] keyBytes) { - Preconditions.checkArgument(keyBytes != null, "Key can't be null"); - 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.aesKey = newKey(keyBytes); + this.cipher = newCipher(); } public byte[] decrypt(byte[] ciphertext, byte[] aad) { @@ -207,6 +183,24 @@ public int decrypt( } } + private static SecretKeySpec newKey(byte[] keyBytes) { + Preconditions.checkArgument(keyBytes != null, "Invalid key: null"); + int keyLength = keyBytes.length; + Preconditions.checkArgument( + (keyLength == 16 || keyLength == 24 || keyLength == 32), + "Invalid key length: %s (must be 16, 24, or 32 bytes)", + keyLength); + return new SecretKeySpec(keyBytes, "AES"); + } + + private static Cipher newCipher() { + try { + return Cipher.getInstance("AES/GCM/NoPadding"); + } catch (GeneralSecurityException e) { + throw new RuntimeException("Failed to create GCM cipher", e); + } + } + static byte[] streamBlockAAD(byte[] fileAadPrefix, int currentBlockIndex) { byte[] blockAAD = ByteBuffer.allocate(4).order(ByteOrder.LITTLE_ENDIAN).putInt(currentBlockIndex).array(); diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java index 0b3f085bc9d7..965c918d6a6e 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -21,10 +21,15 @@ import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; +import java.nio.channels.FileChannel; +import java.nio.file.StandardOpenOption; +import java.util.Arrays; import java.util.Random; +import javax.crypto.AEADBadTagException; import org.apache.iceberg.Files; import org.apache.iceberg.io.PositionOutputStream; import org.apache.iceberg.io.SeekableInputStream; +import org.assertj.core.api.Assertions; import org.junit.Assert; import org.junit.Rule; import org.junit.Test; @@ -54,6 +59,167 @@ public void testEmptyFile() throws IOException { decryptedStream.close(); } + @Test + public void testAADValidation() throws IOException { + Random random = new Random(); + byte[] key = new byte[16]; + random.nextBytes(key); + byte[] aadPrefix = new byte[16]; + random.nextBytes(aadPrefix); + byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block + random.nextBytes(content); + + File testFile = temp.newFile(); + + AesGcmOutputFile encryptedFile = + new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); + try (PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite()) { + encryptedStream.write(content); + } + + // verify the data can be read correctly with the right AAD + AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); + Assert.assertEquals("File size", content.length, decryptedFile.getLength()); + + try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { + byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + int bytesRead = decryptedStream.read(readContent); + Assert.assertEquals("Bytes read should match bytes written", content.length, bytesRead); + Assert.assertEquals( + "Content should match", + ByteBuffer.wrap(content), + ByteBuffer.wrap(readContent, 0, bytesRead)); + } + + // test with the wrong AAD + byte[] badAAD = Arrays.copyOf(aadPrefix, aadPrefix.length); + badAAD[1] -= 1; // modify the AAD slightly + AesGcmInputFile badAADFile = new AesGcmInputFile(Files.localInput(testFile), key, badAAD); + Assert.assertEquals("File size", content.length, badAADFile.getLength()); + + try (SeekableInputStream decryptedStream = badAADFile.newStream()) { + byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent)) + .isInstanceOf(RuntimeException.class) + .hasCauseInstanceOf(AEADBadTagException.class) + .hasMessageContaining("GCM tag check failed"); + } + + // modify the file contents + try (FileChannel out = FileChannel.open(testFile.toPath(), StandardOpenOption.WRITE)) { + long lastTagPosition = testFile.length() - Ciphers.GCM_TAG_LENGTH; + out.position(lastTagPosition); + out.write(ByteBuffer.wrap(key)); // overwrite the tag with other random bytes (the key) + } + + // read with the correct AAD and verify the tag check fails + try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { + byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent)) + .isInstanceOf(RuntimeException.class) + .hasCauseInstanceOf(AEADBadTagException.class) + .hasMessageContaining("GCM tag check failed"); + } + } + + @Test + public void testCorruptNonce() throws IOException { + Random random = new Random(); + byte[] key = new byte[16]; + random.nextBytes(key); + byte[] aadPrefix = new byte[16]; + random.nextBytes(aadPrefix); + byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block + random.nextBytes(content); + + File testFile = temp.newFile(); + + AesGcmOutputFile encryptedFile = + new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); + try (PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite()) { + encryptedStream.write(content); + } + + // verify the data can be read correctly with the right AAD + AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); + Assert.assertEquals("File size", content.length, decryptedFile.getLength()); + + try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { + byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + int bytesRead = decryptedStream.read(readContent); + Assert.assertEquals("Bytes read should match bytes written", content.length, bytesRead); + Assert.assertEquals( + "Content should match", + ByteBuffer.wrap(content), + ByteBuffer.wrap(readContent, 0, bytesRead)); + } + + // replace the first block's nonce + try (FileChannel out = FileChannel.open(testFile.toPath(), StandardOpenOption.WRITE)) { + out.position(Ciphers.GCM_STREAM_HEADER_LENGTH); + // overwrite the nonce with other random bytes (the key) + out.write(ByteBuffer.wrap(key, 0, Ciphers.NONCE_LENGTH)); + } + + // read with the correct AAD and verify the read fails + try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { + byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent)) + .isInstanceOf(RuntimeException.class) + .hasCauseInstanceOf(AEADBadTagException.class) + .hasMessageContaining("GCM tag check failed"); + } + } + + @Test + public void testCorruptCiphertext() throws IOException { + Random random = new Random(); + byte[] key = new byte[16]; + random.nextBytes(key); + byte[] aadPrefix = new byte[16]; + random.nextBytes(aadPrefix); + byte[] content = new byte[Ciphers.PLAIN_BLOCK_SIZE / 2]; // half a block + random.nextBytes(content); + + File testFile = temp.newFile(); + + AesGcmOutputFile encryptedFile = + new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); + try (PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite()) { + encryptedStream.write(content); + } + + // verify the data can be read correctly with the right AAD + AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); + Assert.assertEquals("File size", content.length, decryptedFile.getLength()); + + try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { + byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + int bytesRead = decryptedStream.read(readContent); + Assert.assertEquals("Bytes read should match bytes written", content.length, bytesRead); + Assert.assertEquals( + "Content should match", + ByteBuffer.wrap(content), + ByteBuffer.wrap(readContent, 0, bytesRead)); + } + + // replace part of the first block's content + try (FileChannel out = FileChannel.open(testFile.toPath(), StandardOpenOption.WRITE)) { + out.position(Ciphers.GCM_STREAM_HEADER_LENGTH + Ciphers.NONCE_LENGTH + 34); + // overwrite the nonce with other random bytes (the key) + out.write(ByteBuffer.wrap(key)); + } + + // read with the correct AAD and verify the read fails + try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { + byte[] readContent = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + Assertions.assertThatThrownBy(() -> decryptedStream.read(readContent)) + .isInstanceOf(RuntimeException.class) + .hasCauseInstanceOf(AEADBadTagException.class) + .hasMessageContaining("GCM tag check failed"); + } + } + @Test public void testRandomWriteRead() throws IOException { Random random = new Random(); From 89f30155f79b4691daf7623384751e42b92c2888 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 4 Sep 2023 12:58:16 -0700 Subject: [PATCH 15/16] Support AAD validation for empty files. --- .../iceberg/encryption/AesGcmInputStream.java | 4 +++ .../encryption/AesGcmOutputStream.java | 6 ++--- .../apache/iceberg/encryption/Ciphers.java | 4 +-- .../iceberg/encryption/TestGcmStreams.java | 25 ++++++++++++++++--- 4 files changed, 30 insertions(+), 9 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java index 138dc2dd2a89..57fed69c0172 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -107,6 +107,10 @@ private int availableInCurrentBlock() { public int read(byte[] b, int off, int len) throws IOException { Preconditions.checkArgument(len >= 0, "Invalid read length: " + len); + if (currentPlainBlockIndex < 0) { + decryptBlock(0); + } + if (available() <= 0 && len > 0) { throw new EOFException(); } diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java index ca718603458c..4db0802ea1b3 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -109,9 +109,7 @@ public void close() throws IOException { writeHeader(); } - if (positionInPlainBlock > 0) { - encryptAndWriteBlock(); - } + encryptAndWriteBlock(); targetStream.close(); } @@ -129,7 +127,7 @@ private void encryptAndWriteBlock() throws IOException { throw new IOException("Cannot write block: exceeded Integer.MAX_VALUE blocks"); } - if (positionInPlainBlock == 0) { + if (positionInPlainBlock == 0 && currentBlockIndex != 0) { return; } diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index cf45b99f5c43..c1a5e0f3ee3d 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -80,7 +80,7 @@ public int encrypt( int ciphertextOffset, byte[] aad) { Preconditions.checkArgument( - plaintextLength > 0, "Invalid plain text length: %s", plaintextLength); + plaintextLength >= 0, "Invalid plain text length: %s", plaintextLength); randomGenerator.nextBytes(nonce); int enciphered; @@ -148,7 +148,7 @@ public int decrypt( int plaintextOffset, byte[] aad) { Preconditions.checkState( - ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 1, + ciphertextLength - GCM_TAG_LENGTH - NONCE_LENGTH >= 0, "Cannot decrypt cipher text of length " + ciphertext.length + " because text must longer than GCM_TAG_LENGTH + NONCE_LENGTH bytes. Text may not be encrypted" diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java index 965c918d6a6e..773d5f41af94 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -18,6 +18,7 @@ */ package org.apache.iceberg.encryption; +import java.io.EOFException; import java.io.File; import java.io.IOException; import java.nio.ByteBuffer; @@ -46,17 +47,35 @@ public void testEmptyFile() throws IOException { random.nextBytes(key); byte[] aadPrefix = new byte[16]; random.nextBytes(aadPrefix); + byte[] readBytes = new byte[1]; + File testFile = temp.newFile(); + AesGcmOutputFile encryptedFile = new AesGcmOutputFile(Files.localOutput(testFile), key, aadPrefix); PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); encryptedStream.close(); AesGcmInputFile decryptedFile = new AesGcmInputFile(Files.localInput(testFile), key, aadPrefix); - SeekableInputStream decryptedStream = decryptedFile.newStream(); - Assert.assertEquals("File size", 0, decryptedFile.getLength()); - decryptedStream.close(); + + try (SeekableInputStream decryptedStream = decryptedFile.newStream()) { + Assertions.assertThatThrownBy(() -> decryptedStream.read(readBytes)) + .isInstanceOf(EOFException.class); + } + + // check that the AAD is still verified, even for an empty file + byte[] badAAD = Arrays.copyOf(aadPrefix, aadPrefix.length); + badAAD[1] -= 1; // modify the AAD slightly + AesGcmInputFile badAADFile = new AesGcmInputFile(Files.localInput(testFile), key, badAAD); + Assert.assertEquals("File size", 0, badAADFile.getLength()); + + try (SeekableInputStream decryptedStream = badAADFile.newStream()) { + Assertions.assertThatThrownBy(() -> decryptedStream.read(readBytes)) + .isInstanceOf(RuntimeException.class) + .hasCauseInstanceOf(AEADBadTagException.class) + .hasMessageContaining("GCM tag check failed"); + } } @Test From fcb1ba06c97f4ba7a12241a6501cc0c68f767c10 Mon Sep 17 00:00:00 2001 From: Ryan Blue Date: Mon, 4 Sep 2023 13:15:34 -0700 Subject: [PATCH 16/16] Add check for minimum stream length. --- .../java/org/apache/iceberg/encryption/AesGcmInputFile.java | 4 ++-- core/src/main/java/org/apache/iceberg/encryption/Ciphers.java | 2 ++ 2 files changed, 4 insertions(+), 2 deletions(-) diff --git a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java index ab0e91bcbd61..a43643fcc779 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputFile.java @@ -49,8 +49,8 @@ public long getLength() { public SeekableInputStream newStream() { long ciphertextLength = sourceFile.getLength(); Preconditions.checkState( - ciphertextLength >= Ciphers.GCM_STREAM_HEADER_LENGTH, - "Invalid encrypted stream: %d is shorter than the GCM stream header length", + ciphertextLength >= Ciphers.MIN_STREAM_LENGTH, + "Invalid encrypted stream: %d is shorter than the minimum possible stream length", ciphertextLength); return new AesGcmInputStream(sourceFile.newStream(), ciphertextLength, dataKey, fileAADPrefix); } diff --git a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java index c1a5e0f3ee3d..4aeb1ecad919 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java +++ b/core/src/main/java/org/apache/iceberg/encryption/Ciphers.java @@ -45,6 +45,8 @@ public class Ciphers { private static final int GCM_TAG_LENGTH_BITS = 8 * GCM_TAG_LENGTH; + static final int MIN_STREAM_LENGTH = GCM_STREAM_HEADER_LENGTH + NONCE_LENGTH + GCM_TAG_LENGTH; + private Ciphers() {} public static class AesGcmEncryptor {