diff --git a/.palantir/revapi.yml b/.palantir/revapi.yml index e523814fe3b4..b3a4ba39120c 100644 --- a/.palantir/revapi.yml +++ b/.palantir/revapi.yml @@ -802,6 +802,10 @@ acceptedBreaks: old: "method org.apache.iceberg.view.ViewBuilder org.apache.iceberg.view.ViewBuilder::withQueryColumnNames(java.util.List)" justification: "Acceptable break due to updating View APIs and the View Spec" 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" - code: "java.class.removed" old: "class org.apache.iceberg.actions.BaseDeleteOrphanFilesActionResult" justification: "Removing deprecated code" 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..a43643fcc779 --- /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 org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class AesGcmInputFile implements InputFile { + private final InputFile sourceFile; + private final byte[] dataKey; + private final byte[] fileAADPrefix; + private long plaintextLength; + + public AesGcmInputFile(InputFile sourceFile, byte[] dataKey, byte[] fileAADPrefix) { + this.sourceFile = sourceFile; + this.dataKey = dataKey; + this.fileAADPrefix = fileAADPrefix; + this.plaintextLength = -1; + } + + @Override + public long getLength() { + if (plaintextLength == -1) { + // Presumes all streams use hard-coded plaintext block size. + plaintextLength = AesGcmInputStream.calculatePlaintextLength(sourceFile.getLength()); + } + + return plaintextLength; + } + + @Override + public SeekableInputStream newStream() { + long ciphertextLength = sourceFile.getLength(); + Preconditions.checkState( + 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); + } + + @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 new file mode 100644 index 000000000000..57fed69c0172 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmInputStream.java @@ -0,0 +1,252 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.encryption; + +import java.io.EOFException; +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.iceberg.io.IOUtil; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +public class AesGcmInputStream extends SeekableInputStream { + private final SeekableInputStream sourceStream; + private final byte[] fileAADPrefix; + private final Ciphers.AesGcmDecryptor decryptor; + private final byte[] cipherBlockBuffer; + private final byte[] currentPlainBlock; + private final long numBlocks; + private final int lastCipherBlockSize; + private final long plainStreamSize; + private final byte[] singleByte; + + private long plainStreamPosition; + private long currentPlainBlockIndex; + private int currentPlainBlockSize; + + AesGcmInputStream( + SeekableInputStream sourceStream, long sourceLength, byte[] aesKey, byte[] fileAADPrefix) { + this.sourceStream = sourceStream; + this.fileAADPrefix = fileAADPrefix; + this.decryptor = new Ciphers.AesGcmDecryptor(aesKey); + this.cipherBlockBuffer = new byte[Ciphers.CIPHER_BLOCK_SIZE]; + this.currentPlainBlock = new byte[Ciphers.PLAIN_BLOCK_SIZE]; + this.plainStreamPosition = 0; + this.currentPlainBlockIndex = -1; + this.currentPlainBlockSize = 0; + + long streamLength = sourceLength - Ciphers.GCM_STREAM_HEADER_LENGTH; + long numFullBlocks = Math.toIntExact(streamLength / Ciphers.CIPHER_BLOCK_SIZE); + long cipherFullBlockLength = numFullBlocks * Ciphers.CIPHER_BLOCK_SIZE; + int cipherBytesInLastBlock = Math.toIntExact(streamLength - cipherFullBlockLength); + boolean fullBlocksOnly = (0 == cipherBytesInLastBlock); + this.numBlocks = fullBlocksOnly ? numFullBlocks : numFullBlocks + 1; + this.lastCipherBlockSize = + fullBlocksOnly ? Ciphers.CIPHER_BLOCK_SIZE : cipherBytesInLastBlock; // never 0 + + long lastPlainBlockSize = + (long) lastCipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH; + this.plainStreamSize = + numFullBlocks * Ciphers.PLAIN_BLOCK_SIZE + (fullBlocksOnly ? 0 : lastPlainBlockSize); + this.singleByte = new byte[1]; + } + + private void validateHeader() throws IOException { + byte[] headerBytes = new byte[Ciphers.GCM_STREAM_HEADER_LENGTH]; + IOUtil.readFully(sourceStream, headerBytes, 0, headerBytes.length); + + Preconditions.checkState( + Ciphers.GCM_STREAM_MAGIC.equals(ByteBuffer.wrap(headerBytes, 0, 4)), + "Invalid GCM stream: magic does not match AGS1"); + + int plainBlockSize = ByteBuffer.wrap(headerBytes, 4, 4).order(ByteOrder.LITTLE_ENDIAN).getInt(); + Preconditions.checkState( + plainBlockSize == Ciphers.PLAIN_BLOCK_SIZE, + "Invalid GCM stream: block size %d != %d", + plainBlockSize, + Ciphers.PLAIN_BLOCK_SIZE); + } + + @Override + public int available() { + long maxAvailable = plainStreamSize - plainStreamPosition; + // See InputStream.available contract + if (maxAvailable >= Integer.MAX_VALUE) { + return Integer.MAX_VALUE; + } else { + return (int) maxAvailable; + } + } + + private int availableInCurrentBlock() { + if (blockIndex(plainStreamPosition) != currentPlainBlockIndex) { + return 0; + } + + return currentPlainBlockSize - offsetInBlock(plainStreamPosition); + } + + @Override + 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(); + } + + if (len == 0) { + return 0; + } + + int totalBytesRead = 0; + int resultBufferOffset = off; + int remainingBytesToRead = len; + + while (remainingBytesToRead > 0) { + 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; + } else if (available() > 0) { + decryptBlock(blockIndex(plainStreamPosition)); + + } else { + break; + } + } + + // return -1 for EOF + return totalBytesRead > 0 ? totalBytesRead : -1; + } + + @Override + public void seek(long newPos) throws IOException { + if (newPos < 0) { + throw new IOException("Invalid position: " + newPos); + } else if (newPos > plainStreamSize) { + throw new EOFException( + "Invalid position: " + newPos + " > stream length, " + plainStreamSize); + } + + this.plainStreamPosition = newPos; + } + + @Override + public long skip(long n) { + if (n <= 0) { + return 0; + } + + long bytesLeftInStream = plainStreamSize - plainStreamPosition; + if (n > bytesLeftInStream) { + // skip the rest of the stream + this.plainStreamPosition = plainStreamSize; + return bytesLeftInStream; + } + + this.plainStreamPosition += n; + + return n; + } + + @Override + public long getPos() throws IOException { + return plainStreamPosition; + } + + @Override + public int read() throws IOException { + int read = read(singleByte); + if (read == -1) { + return -1; + } + + return singleByte[0] >= 0 ? singleByte[0] : 256 + singleByte[0]; + } + + @Override + public void close() throws IOException { + sourceStream.close(); + } + + private void decryptBlock(long blockIndex) throws IOException { + if (blockIndex == currentPlainBlockIndex) { + return; + } + + long blockPositionInStream = blockOffset(blockIndex); + if (sourceStream.getPos() != blockPositionInStream) { + if (sourceStream.getPos() == 0) { + validateHeader(); + } + + sourceStream.seek(blockPositionInStream); + } + + boolean isLastBlock = blockIndex == numBlocks - 1; + int cipherBlockSize = isLastBlock ? lastCipherBlockSize : Ciphers.CIPHER_BLOCK_SIZE; + IOUtil.readFully(sourceStream, cipherBlockBuffer, 0, cipherBlockSize); + + byte[] blockAAD = Ciphers.streamBlockAAD(fileAADPrefix, Math.toIntExact(blockIndex)); + decryptor.decrypt(cipherBlockBuffer, 0, cipherBlockSize, currentPlainBlock, 0, blockAAD); + this.currentPlainBlockSize = cipherBlockSize - Ciphers.NONCE_LENGTH - Ciphers.GCM_TAG_LENGTH; + this.currentPlainBlockIndex = blockIndex; + } + + private static long blockIndex(long plainPosition) { + return plainPosition / Ciphers.PLAIN_BLOCK_SIZE; + } + + 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 (numberOfFullBlocks * Ciphers.PLAIN_BLOCK_SIZE) + plainBytesInLastBlock; + } +} 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..c3f5ee2d0327 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputFile.java @@ -0,0 +1,55 @@ +/* + * 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 org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.PositionOutputStream; + +public class AesGcmOutputFile implements OutputFile { + private final OutputFile targetFile; + private final byte[] dataKey; + private final byte[] fileAADPrefix; + + public AesGcmOutputFile(OutputFile targetFile, byte[] dataKey, byte[] fileAADPrefix) { + this.targetFile = targetFile; + this.dataKey = dataKey; + this.fileAADPrefix = fileAADPrefix; + } + + @Override + public PositionOutputStream create() { + return new AesGcmOutputStream(targetFile.create(), dataKey, fileAADPrefix); + } + + @Override + public PositionOutputStream createOrOverwrite() { + return new AesGcmOutputStream(targetFile.createOrOverwrite(), dataKey, fileAADPrefix); + } + + @Override + public String location() { + return targetFile.location(); + } + + @Override + public InputFile toInputFile() { + 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 new file mode 100644 index 000000000000..4db0802ea1b3 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/AesGcmOutputStream.java @@ -0,0 +1,146 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.encryption; + +import java.io.IOException; +import java.nio.ByteBuffer; +import java.nio.ByteOrder; +import org.apache.iceberg.io.PositionOutputStream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; + +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; + private final byte[] singleByte; + private final byte[] plainBlock; + private final byte[] cipherBlock; + + private int positionInPlainBlock; + 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.currentBlockIndex = 0; + this.isHeaderWritten = false; + this.lastBlockWritten = false; + } + + @Override + public void write(int b) throws IOException { + singleByte[0] = (byte) (b & 0x000000FF); + write(singleByte); + } + + @Override + public void write(byte[] b, int off, int len) throws IOException { + if (!isHeaderWritten) { + writeHeader(); + } + + if (b.length - off < len) { + throw new IOException( + "Insufficient bytes in buffer: " + b.length + " - " + off + " < " + len); + } + + int remaining = len; + int offset = off; + + while (remaining > 0) { + int freeBlockBytes = plainBlock.length - positionInPlainBlock; + int toWrite = Math.min(freeBlockBytes, remaining); + + System.arraycopy(b, offset, plainBlock, positionInPlainBlock, toWrite); + positionInPlainBlock += toWrite; + offset += toWrite; + remaining -= toWrite; + + if (positionInPlainBlock == plainBlock.length) { + encryptAndWriteBlock(); + } + } + } + + @Override + public long getPos() throws IOException { + return (long) currentBlockIndex * Ciphers.PLAIN_BLOCK_SIZE + positionInPlainBlock; + } + + @Override + public void flush() throws IOException { + targetStream.flush(); + } + + @Override + public void close() throws IOException { + if (!isHeaderWritten) { + writeHeader(); + } + + encryptAndWriteBlock(); + + targetStream.close(); + } + + 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 && currentBlockIndex != 0) { + 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); + int ciphertextLength = + gcmEncryptor.encrypt(plainBlock, 0, positionInPlainBlock, cipherBlock, 0, aad); + targetStream.write(cipherBlock, 0, ciphertextLength); + positionInPlainBlock = 0; + 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 6eecefda472c..4aeb1ecad919 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,9 @@ */ 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; import javax.crypto.AEADBadTagException; @@ -27,39 +30,61 @@ import org.apache.iceberg.relocated.com.google.common.base.Preconditions; public class Ciphers { - private static final int NONCE_LENGTH = 12; - private static final int GCM_TAG_LENGTH = 16; + public static final int 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 + 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 { 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"); - 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]; } - public byte[] encrypt(byte[] plainText, byte[] aad) { - byte[] nonce = new byte[NONCE_LENGTH]; - randomGenerator.nextBytes(nonce); - int cipherTextLength = NONCE_LENGTH + plainText.length + GCM_TAG_LENGTH; + public byte[] encrypt(byte[] plaintext, byte[] aad) { + return encrypt(plaintext, 0, plaintext.length, aad); + } + + public byte[] encrypt(byte[] plaintext, int plaintextOffset, int plaintextLength, byte[] aad) { + int cipherTextLength = NONCE_LENGTH + plaintextLength + GCM_TAG_LENGTH; byte[] cipherText = new byte[cipherTextLength]; + encrypt(plaintext, plaintextOffset, plaintextLength, cipherText, 0, aad); + return cipherText; + } + + public int encrypt( + byte[] plaintext, + int plaintextOffset, + int plaintextLength, + byte[] ciphertextBuffer, + int ciphertextOffset, + byte[] aad) { + Preconditions.checkArgument( + plaintextLength >= 0, "Invalid plain text length: %s", plaintextLength); + randomGenerator.nextBytes(nonce); + int enciphered; try { GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce); @@ -67,15 +92,32 @@ public byte[] encrypt(byte[] plainText, byte[] aad) { if (null != aad) { cipher.updateAAD(aad); } - cipher.doFinal(plainText, 0, plainText.length, cipherText, NONCE_LENGTH); + + // doFinal encrypts and adds a GCM tag. The nonce is added later. + enciphered = + cipher.doFinal( + plaintext, + plaintextOffset, + plaintextLength, + ciphertextBuffer, + ciphertextOffset + NONCE_LENGTH); + + if (enciphered != plaintextLength + GCM_TAG_LENGTH) { + throw new RuntimeException( + "Failed to encrypt block: expected " + + plaintextLength + + GCM_TAG_LENGTH + + " encrypted bytes but produced bytes " + + enciphered); + } } catch (GeneralSecurityException e) { throw new RuntimeException("Failed to encrypt", e); } // 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; } } @@ -84,54 +126,94 @@ 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) { - int plainTextLength = ciphertext.length - GCM_TAG_LENGTH - NONCE_LENGTH; + return decrypt(ciphertext, 0, ciphertext.length, aad); + } + + public byte[] decrypt( + byte[] ciphertext, int ciphertextOffset, int ciphertextLength, byte[] aad) { + 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( - plainTextLength >= 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" + " with AES GCM cipher"); + int plaintextLength; - // Get the nonce from ciphertext - byte[] nonce = new byte[NONCE_LENGTH]; - System.arraycopy(ciphertext, 0, nonce, 0, NONCE_LENGTH); - - byte[] plainText = new byte[plainTextLength]; - int inputLength = ciphertext.length - NONCE_LENGTH; try { - GCMParameterSpec spec = new GCMParameterSpec(GCM_TAG_LENGTH_BITS, nonce); + 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); } - cipher.doFinal(ciphertext, NONCE_LENGTH, inputLength, plainText, 0); + // For java Cipher, the nonce is not part of ciphertext + 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" - + "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); } - return plainText; + return plaintextLength; + } + } + + 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(); + + 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/TestCiphers.java b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java index a428586c3b46..eedcea4ea2c1 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestCiphers.java @@ -22,24 +22,39 @@ import java.nio.charset.StandardCharsets; import java.security.SecureRandom; +import org.assertj.core.api.Assertions; import org.junit.jupiter.api.Test; 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}; + for (int keyLength : aesKeyLengthArray) { byte[] key = new byte[keyLength]; random.nextBytes(key); @@ -49,8 +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); + + 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"); + } + + if (testCorruption) { + ciphertext[ciphertext.length / 2]++; + + Assertions.assertThatThrownBy(() -> decryptor.decrypt(ciphertext, aad)) + .isInstanceOf(RuntimeException.class) + .hasMessageContaining("GCM tag check failed"); + } } } } 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..773d5f41af94 --- /dev/null +++ b/core/src/test/java/org/apache/iceberg/encryption/TestGcmStreams.java @@ -0,0 +1,416 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.encryption; + +import java.io.EOFException; +import java.io.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; +import org.junit.rules.TemporaryFolder; + +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); + 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); + Assert.assertEquals("File size", 0, decryptedFile.getLength()); + + 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 + 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(); + 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, + 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, aadPrefix); + 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, aadPrefix); + SeekableInputStream decryptedStream = decryptedFile.newStream(); + Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); + + 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); + left = testFileSize - pos; + + if (left < chunkLen) { + chunkLen = left; + } + + decryptedStream.seek(pos); + int len = decryptedStream.read(chunk, 0, chunkLen); + Assert.assertEquals("Read length", len, chunkLen); + 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(); + } + } + } + + @Test + 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 + }; + + 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, aadPrefix); + PositionOutputStream encryptedStream = encryptedFile.createOrOverwrite(); + + int offset = 0; + int chunkLen = Ciphers.PLAIN_BLOCK_SIZE; + 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, aadPrefix); + SeekableInputStream decryptedStream = decryptedFile.newStream(); + Assert.assertEquals("File size", testFileSize, decryptedFile.getLength()); + + offset = 0; + chunkLen = Ciphers.PLAIN_BLOCK_SIZE; + byte[] chunk = new byte[chunkLen]; + left = testFileSize; + + while (left > 0) { + + if (chunkLen > left) { + chunkLen = left; + } + + 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, offset, chunkLen); + Assert.assertEquals("Read contents", bb1, bb2); + + offset += len; + left = testFileSize - offset; + } + + decryptedStream.close(); + } + } +}