diff --git a/api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java b/api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java index 1686342c776d..300c88d18862 100644 --- a/api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java +++ b/api/src/main/java/org/apache/iceberg/encryption/EncryptedOutputFile.java @@ -37,4 +37,9 @@ public interface EncryptedOutputFile { * #encryptingOutputFile()}. */ EncryptionKeyMetadata keyMetadata(); + + /** Underlying output file for native encryption. */ + default OutputFile plainOutputFile() { + throw new UnsupportedOperationException("Not implemented"); + } } diff --git a/core/src/main/java/org/apache/iceberg/CatalogProperties.java b/core/src/main/java/org/apache/iceberg/CatalogProperties.java index cada23437528..b6fd990f0ac6 100644 --- a/core/src/main/java/org/apache/iceberg/CatalogProperties.java +++ b/core/src/main/java/org/apache/iceberg/CatalogProperties.java @@ -156,4 +156,7 @@ private CatalogProperties() {} public static final String AUTH_SESSION_TIMEOUT_MS = "auth.session-timeout-ms"; public static final long AUTH_SESSION_TIMEOUT_MS_DEFAULT = TimeUnit.HOURS.toMillis(1); + + public static final String ENCRYPTION_KMS_TYPE = "encryption.kms-type"; + public static final String ENCRYPTION_KMS_IMPL = "encryption.kms-impl"; } diff --git a/core/src/main/java/org/apache/iceberg/TableOperations.java b/core/src/main/java/org/apache/iceberg/TableOperations.java index 6822809d799e..7f3106253ca6 100644 --- a/core/src/main/java/org/apache/iceberg/TableOperations.java +++ b/core/src/main/java/org/apache/iceberg/TableOperations.java @@ -71,7 +71,7 @@ public interface TableOperations { * files. */ default EncryptionManager encryption() { - return new PlaintextEncryptionManager(); + return PlaintextEncryptionManager.instance(); } /** diff --git a/core/src/main/java/org/apache/iceberg/TableProperties.java b/core/src/main/java/org/apache/iceberg/TableProperties.java index 03e1f3ce8897..fb5da6065839 100644 --- a/core/src/main/java/org/apache/iceberg/TableProperties.java +++ b/core/src/main/java/org/apache/iceberg/TableProperties.java @@ -369,4 +369,11 @@ private TableProperties() {} public static final String UPSERT_ENABLED = "write.upsert.enabled"; public static final boolean UPSERT_ENABLED_DEFAULT = false; + + public static final String ENCRYPTION_TABLE_KEY = "encryption.key-id"; + + public static final String ENCRYPTION_DEK_LENGTH = "encryption.data-key-length"; + public static final int ENCRYPTION_DEK_LENGTH_DEFAULT = 16; + + public static final int ENCRYPTION_AAD_LENGTH_DEFAULT = 16; } diff --git a/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java b/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java new file mode 100644 index 000000000000..ad1deecb8da1 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java @@ -0,0 +1,107 @@ +/* + * 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.util.Map; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.FileFormat; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.common.DynConstructors; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.util.PropertyUtil; + +public class EncryptionUtil { + + private EncryptionUtil() {} + + public static KeyManagementClient createKmsClient(Map catalogProperties) { + String kmsType = catalogProperties.get(CatalogProperties.ENCRYPTION_KMS_TYPE); + String kmsImpl = catalogProperties.get(CatalogProperties.ENCRYPTION_KMS_IMPL); + + Preconditions.checkArgument( + kmsType == null || kmsImpl == null, + "Cannot set both KMS type (%s) and KMS impl (%s)", + kmsType, + kmsImpl); + + // TODO: Add KMS implementations + Preconditions.checkArgument(kmsType == null, "Unsupported KMS type: %s", kmsType); + + KeyManagementClient kmsClient; + DynConstructors.Ctor ctor; + try { + ctor = DynConstructors.builder(KeyManagementClient.class).impl(kmsImpl).buildChecked(); + } catch (NoSuchMethodException e) { + throw new IllegalArgumentException( + String.format( + "Cannot initialize KeyManagementClient, missing no-arg constructor for class %s", + kmsImpl), + e); + } + + try { + kmsClient = ctor.newInstance(); + } catch (ClassCastException e) { + throw new IllegalArgumentException( + String.format( + "Cannot initialize kms client, %s does not implement KeyManagementClient interface", + kmsImpl), + e); + } + + kmsClient.initialize(catalogProperties); + + return kmsClient; + } + + public static EncryptionManager createEncryptionManager( + Map tableProperties, KeyManagementClient kmsClient) { + Preconditions.checkArgument(kmsClient != null, "Invalid KMS client: null"); + String tableKeyId = tableProperties.get(TableProperties.ENCRYPTION_TABLE_KEY); + + if (null == tableKeyId) { + // Unencrypted table + return PlaintextEncryptionManager.instance(); + } + + String fileFormat = + PropertyUtil.propertyAsString( + tableProperties, + TableProperties.DEFAULT_FILE_FORMAT, + TableProperties.DEFAULT_FILE_FORMAT_DEFAULT); + + if (FileFormat.fromString(fileFormat) != FileFormat.PARQUET) { + throw new UnsupportedOperationException( + "Iceberg encryption currently supports only parquet format for data files"); + } + + int dataKeyLength = + PropertyUtil.propertyAsInt( + tableProperties, + TableProperties.ENCRYPTION_DEK_LENGTH, + TableProperties.ENCRYPTION_DEK_LENGTH_DEFAULT); + + Preconditions.checkState( + dataKeyLength == 16 || dataKeyLength == 24 || dataKeyLength == 32, + "Invalid data key length: %s (must be 16, 24, or 32)", + dataKeyLength); + + return new StandardEncryptionManager(tableKeyId, dataKeyLength, kmsClient); + } +} diff --git a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java index 674685c30164..7e57163d73ea 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java +++ b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataDecoder.java @@ -28,9 +28,9 @@ import org.apache.iceberg.data.avro.RawDecoder; import org.apache.iceberg.relocated.com.google.common.collect.MapMaker; -class KeyMetadataDecoder extends MessageDecoder.BaseDecoder { +class KeyMetadataDecoder extends MessageDecoder.BaseDecoder { private final org.apache.iceberg.Schema readSchema; - private final Map> decoders = new MapMaker().makeMap(); + private final Map> decoders = new MapMaker().makeMap(); /** * Creates a new decoder that constructs key metadata instances described by schema version. @@ -39,11 +39,11 @@ class KeyMetadataDecoder extends MessageDecoder.BaseDecoder { * instances created by this class will are described by the expected schema. */ KeyMetadataDecoder(byte readSchemaVersion) { - this.readSchema = KeyMetadata.supportedSchemaVersions().get(readSchemaVersion); + this.readSchema = StandardKeyMetadata.supportedSchemaVersions().get(readSchemaVersion); } @Override - public KeyMetadata decode(InputStream stream, KeyMetadata reuse) { + public StandardKeyMetadata decode(InputStream stream, StandardKeyMetadata reuse) { byte writeSchemaVersion; try { @@ -56,14 +56,14 @@ public KeyMetadata decode(InputStream stream, KeyMetadata reuse) { throw new RuntimeException("Version byte - end of stream reached"); } - Schema writeSchema = KeyMetadata.supportedAvroSchemaVersions().get(writeSchemaVersion); + Schema writeSchema = StandardKeyMetadata.supportedAvroSchemaVersions().get(writeSchemaVersion); if (writeSchema == null) { throw new UnsupportedOperationException( "Cannot resolve schema for version: " + writeSchemaVersion); } - RawDecoder decoder = decoders.get(writeSchemaVersion); + RawDecoder decoder = decoders.get(writeSchemaVersion); if (decoder == null) { decoder = new RawDecoder<>(readSchema, GenericAvroReader::create, writeSchema); diff --git a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataEncoder.java b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataEncoder.java index faab6a47c814..6f9d1769f07d 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataEncoder.java +++ b/core/src/main/java/org/apache/iceberg/encryption/KeyMetadataEncoder.java @@ -29,18 +29,18 @@ import org.apache.avro.message.MessageEncoder; import org.apache.iceberg.avro.GenericAvroWriter; -class KeyMetadataEncoder implements MessageEncoder { +class KeyMetadataEncoder implements MessageEncoder { private static final ThreadLocal TEMP = ThreadLocal.withInitial(BufferOutputStream::new); private static final ThreadLocal ENCODER = new ThreadLocal<>(); private final byte schemaVersion; private final boolean copyOutputBytes; - private final DatumWriter writer; + private final DatumWriter writer; /** - * Creates a new {@link MessageEncoder} that will deconstruct {@link KeyMetadata} instances - * described by the schema version. + * Creates a new {@link MessageEncoder} that will deconstruct {@link StandardKeyMetadata} + * instances described by the schema version. * *

Buffers returned by {@code encode} are copied and will not be modified by future calls to * {@code encode}. @@ -50,8 +50,8 @@ class KeyMetadataEncoder implements MessageEncoder { } /** - * Creates a new {@link MessageEncoder} that will deconstruct {@link KeyMetadata} instances - * described by the schema version. + * Creates a new {@link MessageEncoder} that will deconstruct {@link StandardKeyMetadata} + * instances described by the schema version. * *

If {@code shouldCopy} is true, then buffers returned by {@code encode} are copied and will * not be modified by future calls to {@code encode}. @@ -62,7 +62,7 @@ class KeyMetadataEncoder implements MessageEncoder { * next call to {@code encode}. */ KeyMetadataEncoder(byte schemaVersion, boolean shouldCopy) { - Schema writeSchema = KeyMetadata.supportedAvroSchemaVersions().get(schemaVersion); + Schema writeSchema = StandardKeyMetadata.supportedAvroSchemaVersions().get(schemaVersion); if (writeSchema == null) { throw new UnsupportedOperationException( @@ -75,7 +75,7 @@ class KeyMetadataEncoder implements MessageEncoder { } @Override - public ByteBuffer encode(KeyMetadata datum) throws IOException { + public ByteBuffer encode(StandardKeyMetadata datum) throws IOException { BufferOutputStream temp = TEMP.get(); temp.reset(); temp.write(schemaVersion); @@ -89,7 +89,7 @@ public ByteBuffer encode(KeyMetadata datum) throws IOException { } @Override - public void encode(KeyMetadata datum, OutputStream stream) throws IOException { + public void encode(StandardKeyMetadata datum, OutputStream stream) throws IOException { BinaryEncoder encoder = EncoderFactory.get().directBinaryEncoder(stream, ENCODER.get()); ENCODER.set(encoder); writer.write(datum, encoder); diff --git a/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java b/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java index 4d8d8aa7aff9..336ebe9c0af7 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java +++ b/core/src/main/java/org/apache/iceberg/encryption/PlaintextEncryptionManager.java @@ -18,26 +18,33 @@ */ package org.apache.iceberg.encryption; -import java.nio.ByteBuffer; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.OutputFile; import org.slf4j.Logger; import org.slf4j.LoggerFactory; public class PlaintextEncryptionManager implements EncryptionManager { + private static final EncryptionManager INSTANCE = new PlaintextEncryptionManager(); private static final Logger LOG = LoggerFactory.getLogger(PlaintextEncryptionManager.class); + /** @deprecated will be removed in 1.6.0. use {@link #instance()} instead. */ + @Deprecated + public PlaintextEncryptionManager() {} + + public static EncryptionManager instance() { + return INSTANCE; + } + @Override public InputFile decrypt(EncryptedInputFile encrypted) { if (encrypted.keyMetadata().buffer() != null) { - LOG.warn( - "File encryption key metadata is present, but currently using PlaintextEncryptionManager."); + LOG.warn("File encryption key metadata is present, but no encryption has been configured."); } return encrypted.encryptedInputFile(); } @Override public EncryptedOutputFile encrypt(OutputFile rawOutput) { - return EncryptedFiles.encryptedOutput(rawOutput, (ByteBuffer) null); + return EncryptedFiles.encryptedOutput(rawOutput, EncryptionKeyMetadata.empty()); } } diff --git a/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java new file mode 100644 index 000000000000..63f89e7661b3 --- /dev/null +++ b/core/src/main/java/org/apache/iceberg/encryption/StandardEncryptionManager.java @@ -0,0 +1,193 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, + * software distributed under the License is distributed on an + * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY + * KIND, either express or implied. See the License for the + * specific language governing permissions and limitations + * under the License. + */ +package org.apache.iceberg.encryption; + +import java.nio.ByteBuffer; +import java.security.SecureRandom; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SeekableInputStream; +import org.apache.iceberg.relocated.com.google.common.base.Preconditions; +import org.apache.iceberg.relocated.com.google.common.collect.Iterables; +import org.apache.iceberg.util.ByteBuffers; + +public class StandardEncryptionManager implements EncryptionManager { + private final transient KeyManagementClient kmsClient; + private final String tableKeyId; + private final int dataKeyLength; + + private transient volatile SecureRandom lazyRNG = null; + + /** + * @param tableKeyId table encryption key id + * @param dataKeyLength length of data encryption key (16/24/32 bytes) + * @param kmsClient Client of KMS used to wrap/unwrap keys in envelope encryption + */ + public StandardEncryptionManager( + String tableKeyId, int dataKeyLength, KeyManagementClient kmsClient) { + Preconditions.checkNotNull(tableKeyId, "Invalid encryption key ID: null"); + Preconditions.checkArgument( + dataKeyLength == 16 || dataKeyLength == 24 || dataKeyLength == 32, + "Invalid data key length: %s (must be 16, 24, or 32)", + dataKeyLength); + Preconditions.checkNotNull(kmsClient, "Invalid KMS client: null"); + this.tableKeyId = tableKeyId; + this.kmsClient = kmsClient; + this.dataKeyLength = dataKeyLength; + } + + @Override + public EncryptedOutputFile encrypt(OutputFile plainOutput) { + return new StandardEncryptedOutputFile(plainOutput, dataKeyLength); + } + + @Override + public InputFile decrypt(EncryptedInputFile encrypted) { + // this input file will lazily parse key metadata in case the file is not an AES GCM stream. + return new StandardDecryptedInputFile(encrypted); + } + + @Override + public Iterable decrypt(Iterable encrypted) { + // Bulk decrypt is only applied to data files. Returning source input files for parquet. + return Iterables.transform(encrypted, this::decrypt); + } + + private SecureRandom workerRNG() { + if (this.lazyRNG == null) { + this.lazyRNG = new SecureRandom(); + } + + return lazyRNG; + } + + public ByteBuffer wrapKey(ByteBuffer secretKey) { + if (kmsClient == null) { + throw new IllegalStateException( + "Cannot wrap key after called after serialization (missing KMS client)"); + } + + return kmsClient.wrapKey(secretKey, tableKeyId); + } + + public ByteBuffer unwrapKey(ByteBuffer wrappedSecretKey) { + if (kmsClient == null) { + throw new IllegalStateException( + "Cannot wrap key after called after serialization (missing KMS client)"); + } + + return kmsClient.unwrapKey(wrappedSecretKey, tableKeyId); + } + + private class StandardEncryptedOutputFile implements EncryptedOutputFile { + private final OutputFile plainOutputFile; + private final int dataKeyLength; + private StandardKeyMetadata lazyKeyMetadata = null; + private OutputFile lazyEncryptingOutputFile = null; + + StandardEncryptedOutputFile(OutputFile plainOutputFile, int dataKeyLength) { + this.plainOutputFile = plainOutputFile; + this.dataKeyLength = dataKeyLength; + } + + @Override + public StandardKeyMetadata keyMetadata() { + if (null == lazyKeyMetadata) { + byte[] fileDek = new byte[dataKeyLength]; + workerRNG().nextBytes(fileDek); + + byte[] aadPrefix = new byte[TableProperties.ENCRYPTION_AAD_LENGTH_DEFAULT]; + workerRNG().nextBytes(aadPrefix); + + this.lazyKeyMetadata = new StandardKeyMetadata(fileDek, aadPrefix); + } + + return lazyKeyMetadata; + } + + @Override + public OutputFile encryptingOutputFile() { + if (null == lazyEncryptingOutputFile) { + this.lazyEncryptingOutputFile = + new AesGcmOutputFile( + plainOutputFile, + ByteBuffers.toByteArray(keyMetadata().encryptionKey()), + ByteBuffers.toByteArray(keyMetadata().aadPrefix())); + } + + return lazyEncryptingOutputFile; + } + + @Override + public OutputFile plainOutputFile() { + return plainOutputFile; + } + } + + private static class StandardDecryptedInputFile implements InputFile { + private final EncryptedInputFile encryptedInputFile; + private StandardKeyMetadata lazyKeyMetadata = null; + private AesGcmInputFile lazyDecryptedInputFile = null; + + private StandardDecryptedInputFile(EncryptedInputFile encryptedInputFile) { + this.encryptedInputFile = encryptedInputFile; + } + + private StandardKeyMetadata keyMetadata() { + if (null == lazyKeyMetadata) { + this.lazyKeyMetadata = StandardKeyMetadata.castOrParse(encryptedInputFile.keyMetadata()); + } + + return lazyKeyMetadata; + } + + private AesGcmInputFile decrypted() { + if (null == lazyDecryptedInputFile) { + this.lazyDecryptedInputFile = + new AesGcmInputFile( + encryptedInputFile.encryptedInputFile(), + ByteBuffers.toByteArray(keyMetadata().encryptionKey()), + ByteBuffers.toByteArray(keyMetadata().aadPrefix())); + } + + return lazyDecryptedInputFile; + } + + @Override + public long getLength() { + return decrypted().getLength(); + } + + @Override + public SeekableInputStream newStream() { + return decrypted().newStream(); + } + + @Override + public String location() { + return decrypted().location(); + } + + @Override + public boolean exists() { + return decrypted().exists(); + } + } +} diff --git a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadata.java b/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java similarity index 80% rename from core/src/main/java/org/apache/iceberg/encryption/KeyMetadata.java rename to core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java index 0d7ec43f6ebc..2ac70aebc316 100644 --- a/core/src/main/java/org/apache/iceberg/encryption/KeyMetadata.java +++ b/core/src/main/java/org/apache/iceberg/encryption/StandardKeyMetadata.java @@ -31,14 +31,14 @@ import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.types.Types; -class KeyMetadata implements EncryptionKeyMetadata, IndexedRecord { +class StandardKeyMetadata implements EncryptionKeyMetadata, IndexedRecord { private static final byte V1 = 1; private static final Schema SCHEMA_V1 = new Schema( required(0, "encryption_key", Types.BinaryType.get()), optional(1, "aad_prefix", Types.BinaryType.get())); private static final org.apache.avro.Schema AVRO_SCHEMA_V1 = - AvroSchemaUtil.convert(SCHEMA_V1, KeyMetadata.class.getCanonicalName()); + AvroSchemaUtil.convert(SCHEMA_V1, StandardKeyMetadata.class.getCanonicalName()); private static final Map schemaVersions = ImmutableMap.of(V1, SCHEMA_V1); private static final Map avroSchemaVersions = @@ -52,9 +52,14 @@ class KeyMetadata implements EncryptionKeyMetadata, IndexedRecord { private org.apache.avro.Schema avroSchema; /** Used by Avro reflection to instantiate this class * */ - KeyMetadata() {} + StandardKeyMetadata() {} - KeyMetadata(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { + StandardKeyMetadata(byte[] key, byte[] aad) { + this.encryptionKey = ByteBuffer.wrap(key); + this.aadPrefix = ByteBuffer.wrap(aad); + } + + private StandardKeyMetadata(ByteBuffer encryptionKey, ByteBuffer aadPrefix) { this.encryptionKey = encryptionKey; this.aadPrefix = aadPrefix; this.avroSchema = AVRO_SCHEMA_V1; @@ -76,7 +81,21 @@ ByteBuffer aadPrefix() { return aadPrefix; } - static KeyMetadata parse(ByteBuffer buffer) { + static StandardKeyMetadata castOrParse(EncryptionKeyMetadata keyMetadata) { + if (keyMetadata instanceof StandardKeyMetadata) { + return (StandardKeyMetadata) keyMetadata; + } + + ByteBuffer kmBuffer = keyMetadata.buffer(); + + if (kmBuffer == null) { + throw new IllegalStateException("Null key metadata buffer"); + } + + return parse(kmBuffer); + } + + static StandardKeyMetadata parse(ByteBuffer buffer) { try { return KEY_METADATA_DECODER.decode(buffer); } catch (IOException e) { @@ -95,8 +114,7 @@ public ByteBuffer buffer() { @Override public EncryptionKeyMetadata copy() { - KeyMetadata metadata = new KeyMetadata(encryptionKey(), aadPrefix()); - return metadata; + return new StandardKeyMetadata(encryptionKey(), aadPrefix()); } @Override diff --git a/core/src/test/java/org/apache/iceberg/encryption/TestKeyMetadataParser.java b/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java similarity index 84% rename from core/src/test/java/org/apache/iceberg/encryption/TestKeyMetadataParser.java rename to core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java index 27b00b1829c6..889506cb93e4 100644 --- a/core/src/test/java/org/apache/iceberg/encryption/TestKeyMetadataParser.java +++ b/core/src/test/java/org/apache/iceberg/encryption/TestStandardKeyMetadataParser.java @@ -24,16 +24,17 @@ import org.junit.Assert; import org.junit.Test; -public class TestKeyMetadataParser { +public class TestStandardKeyMetadataParser { @Test public void testParser() { ByteBuffer encryptionKey = ByteBuffer.wrap("0123456789012345".getBytes(StandardCharsets.UTF_8)); ByteBuffer aadPrefix = ByteBuffer.wrap("1234567890123456".getBytes(StandardCharsets.UTF_8)); - KeyMetadata metadata = new KeyMetadata(encryptionKey, aadPrefix); + StandardKeyMetadata metadata = + new StandardKeyMetadata(encryptionKey.array(), aadPrefix.array()); ByteBuffer serialized = metadata.buffer(); - KeyMetadata parsedMetadata = KeyMetadata.parse(serialized); + StandardKeyMetadata parsedMetadata = StandardKeyMetadata.parse(serialized); Assert.assertEquals(parsedMetadata.encryptionKey(), encryptionKey); Assert.assertEquals(parsedMetadata.aadPrefix(), aadPrefix); } @@ -41,7 +42,7 @@ public void testParser() { @Test public void testUnsupportedVersion() { ByteBuffer badBuffer = ByteBuffer.wrap(new byte[] {0x02}); - Assertions.assertThatThrownBy(() -> KeyMetadata.parse(badBuffer)) + Assertions.assertThatThrownBy(() -> StandardKeyMetadata.parse(badBuffer)) .isInstanceOf(UnsupportedOperationException.class) .hasMessage("Cannot resolve schema for version: 2"); }