Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

Standard key manager #6884

Closed
wants to merge 13 commits into from
Original file line number Diff line number Diff line change
Expand Up @@ -37,4 +37,9 @@ public interface EncryptedOutputFile {
* #encryptingOutputFile()}.
*/
EncryptionKeyMetadata keyMetadata();

/** Underlying output file for native encryption. */
default OutputFile rawOutputFile() {
Copy link
Contributor

@rdblue rdblue Dec 10, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I'm not sure about this name, but I've had a hard time coming up with a better one. The best I can come up with is plainOutputFile. What do you think about that? I think it's slightly clearer than "raw".

It also looks like this isn't used in this PR, in which case I think we should include it in a PR where it is used.

throw new UnsupportedOperationException("Not implemented");
};
}
3 changes: 3 additions & 0 deletions core/src/main/java/org/apache/iceberg/CatalogProperties.java
Original file line number Diff line number Diff line change
Expand Up @@ -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_CUSTOM_TYPE = "custom";
public static final String ENCRYPTION_KMS_CLIENT_IMPL = "encryption.kms.client-impl";
}
2 changes: 1 addition & 1 deletion core/src/main/java/org/apache/iceberg/TableOperations.java
Original file line number Diff line number Diff line change
Expand Up @@ -71,7 +71,7 @@ public interface TableOperations {
* files.
*/
default EncryptionManager encryption() {
return new PlaintextEncryptionManager();
return PlaintextEncryptionManager.instance();
}

/**
Expand Down
7 changes: 7 additions & 0 deletions core/src/main/java/org/apache/iceberg/TableProperties.java
Original file line number Diff line number Diff line change
Expand Up @@ -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;
}
Original file line number Diff line number Diff line change
Expand Up @@ -57,5 +57,9 @@ public static EncryptedOutputFile encryptedOutput(
encryptedOutputFile, BaseEncryptionKeyMetadata.fromByteArray(keyMetadata));
}

public static EncryptedOutputFile plainAsEncryptedOutput(OutputFile encryptingOutputFile) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This isn't used. Can we add it when the use is added?

return new BaseEncryptedOutputFile(encryptingOutputFile, EncryptionKeyMetadata.empty());
}

private EncryptedFiles() {}
}
137 changes: 137 additions & 0 deletions core/src/main/java/org/apache/iceberg/encryption/EncryptionUtil.java
Original file line number Diff line number Diff line change
@@ -0,0 +1,137 @@
/*
* 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 static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT;
import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT_DEFAULT;
import static org.apache.iceberg.TableProperties.ENCRYPTION_DEK_LENGTH;
import static org.apache.iceberg.TableProperties.ENCRYPTION_DEK_LENGTH_DEFAULT;
import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY;

import java.nio.ByteBuffer;
import java.util.Map;
import org.apache.iceberg.CatalogProperties;
import org.apache.iceberg.FileFormat;
import org.apache.iceberg.common.DynConstructors;
import org.apache.iceberg.util.PropertyUtil;

public class EncryptionUtil {

private EncryptionUtil() {}

public static KeyMetadata parseKeyMetadata(ByteBuffer metadataBuffer) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This isn't used and duplicates the wrapped method. I think we should remove it.

return KeyMetadata.parse(metadataBuffer);
}

public static EncryptionKeyMetadata createKeyMetadata(ByteBuffer key, ByteBuffer aadPrefix) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This isn't used and it looks like a few places use the KeyMetadata constructor directly.

return new KeyMetadata(key, aadPrefix);
}

public static long gcmEncryptionLength(long plainFileLength) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

Can we remove this since it isn't used? I'd rather not have dead and untested code.

int numberOfFullBlocks = Math.toIntExact(plainFileLength / Ciphers.PLAIN_BLOCK_SIZE);
int plainBytesInLastBlock =
Math.toIntExact(plainFileLength - numberOfFullBlocks * Ciphers.PLAIN_BLOCK_SIZE);
boolean fullBlocksOnly = (0 == plainBytesInLastBlock);
int cipherBytesInLastBlock =
fullBlocksOnly ? 0 : plainBytesInLastBlock + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
int cipherBlockSize = Ciphers.PLAIN_BLOCK_SIZE + Ciphers.NONCE_LENGTH + Ciphers.GCM_TAG_LENGTH;
return (long) Ciphers.GCM_STREAM_HEADER_LENGTH
+ numberOfFullBlocks * cipherBlockSize
+ cipherBytesInLastBlock;
}

public static KeyManagementClient createKmsClient(Map<String, String> catalogProperties) {
String kmsType = catalogProperties.get(CatalogProperties.ENCRYPTION_KMS_TYPE);

if (kmsType == null) {
throw new IllegalStateException(
"Cannot create StandardEncryptionManagerFactory without KMS type");
}

if (!kmsType.equals(CatalogProperties.ENCRYPTION_KMS_CUSTOM_TYPE)) {
Copy link
Contributor

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

I think that it is much simpler if rather than requiring a type that is "custom", this instead checks that either type or impl is set.

// Currently support only custom types
throw new UnsupportedOperationException("Undefined KMS type " + kmsType);
}

String kmsClientImpl = catalogProperties.get(CatalogProperties.ENCRYPTION_KMS_CLIENT_IMPL);

if (kmsClientImpl == null) {
throw new IllegalStateException("Custom KMS client class is not defined");
}

KeyManagementClient kmsClient;
DynConstructors.Ctor<KeyManagementClient> ctor;
try {
ctor = DynConstructors.builder(KeyManagementClient.class).impl(kmsClientImpl).buildChecked();
} catch (NoSuchMethodException e) {
throw new IllegalArgumentException(
String.format(
"Cannot initialize KeyManagementClient, missing no-arg constructor for class %s",
kmsClientImpl),
e);
}

try {
kmsClient = ctor.newInstance();
} catch (ClassCastException e) {
throw new IllegalArgumentException(
String.format(
"Cannot initialize kms client, %s does not implement KeyManagementClient interface",
kmsClientImpl),
e);
}

kmsClient.initialize(catalogProperties);

return kmsClient;
}

public static EncryptionManager createEncryptionManager(
Map<String, String> tableProperties, KeyManagementClient kmsClient) {
String tableKeyId = tableProperties.get(ENCRYPTION_TABLE_KEY);

if (null == tableKeyId) {
// Unencrypted table
return PlaintextEncryptionManager.instance();
}

if (kmsClient == null) {
throw new IllegalStateException("Encrypted table. No KMS client is configured in catalog");
}

String fileFormat =
PropertyUtil.propertyAsString(
tableProperties, DEFAULT_FILE_FORMAT, 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, ENCRYPTION_DEK_LENGTH, ENCRYPTION_DEK_LENGTH_DEFAULT);

return new StandardEncryptionManager(tableKeyId, dataKeyLength, kmsClient);
}

public static boolean useNativeEncryption(EncryptionKeyMetadata keyMetadata) {
return keyMetadata != null && keyMetadata instanceof KeyMetadata;
}
}
14 changes: 14 additions & 0 deletions core/src/main/java/org/apache/iceberg/encryption/KeyMetadata.java
Original file line number Diff line number Diff line change
Expand Up @@ -76,6 +76,20 @@ ByteBuffer aadPrefix() {
return aadPrefix;
}

static KeyMetadata castOrParse(EncryptionKeyMetadata keyMetadata) {
if (keyMetadata instanceof KeyMetadata) {
return (KeyMetadata) keyMetadata;
}

ByteBuffer kmBuffer = keyMetadata.buffer();

if (kmBuffer == null) {
throw new IllegalStateException("Null key metadata buffer");
}

return parse(kmBuffer);
}

static KeyMetadata parse(ByteBuffer buffer) {
try {
return KEY_METADATA_DECODER.decode(buffer);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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();
rdblue marked this conversation as resolved.
Show resolved Hide resolved
private static final Logger LOG = LoggerFactory.getLogger(PlaintextEncryptionManager.class);

/** @deprecated will be removed in 1.6.0. use {@link #instance()} instead. */
@Deprecated
rdblue marked this conversation as resolved.
Show resolved Hide resolved
public PlaintextEncryptionManager() {}

public static EncryptionManager instance() {
rdblue marked this conversation as resolved.
Show resolved Hide resolved
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());
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,142 @@
/*
* 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.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;

class StandardEncryptedOutputFile implements EncryptedOutputFile {

private final OutputFile encryptingOutputFile;

private final EncryptionKeyMetadata keyMetadata;
private final OutputFile rawOutputFile;

StandardEncryptedOutputFile(
OutputFile encryptingOutputFile,
EncryptionKeyMetadata keyMetadata,
OutputFile rawOutputFile) {
this.encryptingOutputFile = encryptingOutputFile;
this.keyMetadata = keyMetadata;
this.rawOutputFile = rawOutputFile;
}

@Override
public OutputFile encryptingOutputFile() {
return encryptingOutputFile;
}

@Override
public EncryptionKeyMetadata keyMetadata() {
return keyMetadata;
}

@Override
public OutputFile rawOutputFile() {
return rawOutputFile;
}
}

/**
* @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.checkNotNull(kmsClient, "Invalid KMS client: null");
this.tableKeyId = tableKeyId;
this.kmsClient = kmsClient;
this.dataKeyLength = dataKeyLength;
}

@Override
public EncryptedOutputFile encrypt(OutputFile rawOutput) {
ByteBuffer fileDek = ByteBuffer.allocate(dataKeyLength);
workerRNG().nextBytes(fileDek.array());

ByteBuffer aadPrefix = ByteBuffer.allocate(TableProperties.ENCRYPTION_AAD_LENGTH_DEFAULT);
workerRNG().nextBytes(aadPrefix.array());

KeyMetadata encryptionMetadata = new KeyMetadata(fileDek, aadPrefix);

return new StandardEncryptedOutputFile(
new AesGcmOutputFile(rawOutput, fileDek.array(), aadPrefix.array()),
encryptionMetadata,
rawOutput);
}

@Override
public InputFile decrypt(EncryptedInputFile encrypted) {
KeyMetadata keyMetadata = KeyMetadata.castOrParse(encrypted.keyMetadata());

byte[] fileDek = ByteBuffers.toByteArray(keyMetadata.encryptionKey());
byte[] aadPrefix = ByteBuffers.toByteArray(keyMetadata.aadPrefix());

return new AesGcmInputFile(encrypted.encryptedInputFile(), fileDek, aadPrefix);
}

@Override
public Iterable<InputFile> decrypt(Iterable<EncryptedInputFile> encrypted) {
// Bulk decrypt is only applied to data files. Returning source input files for parquet.
return Iterables.transform(encrypted, this::getSourceFile);
Copy link
Contributor

@rdblue rdblue Dec 10, 2023

Choose a reason for hiding this comment

The reason will be displayed to describe this comment to others. Learn more.

This is incorrect. Based on the comment, I think the intent was to skip creating an AesGcmInputFile because you know that this is only called for data files. But that's a bad assumption and there is no guarantee it will be called that way in the future.

Instead, I think this should call decrypt. That operation should be cheap, although we may need to defer parsing the key metadata until later.

}

private InputFile getSourceFile(EncryptedInputFile encryptedFile) {
return encryptedFile.encryptedInputFile();
}

private SecureRandom workerRNG() {
if (this.lazyRNG == null) {
this.lazyRNG = new SecureRandom();
}

return lazyRNG;
}

public ByteBuffer wrapKey(ByteBuffer secretKey) {
if (kmsClient == null) {
throw new IllegalStateException("Null KmsClient. WrapKey can't be called from workers");
}

return kmsClient.wrapKey(secretKey, tableKeyId);
rdblue marked this conversation as resolved.
Show resolved Hide resolved
}

public ByteBuffer unwrapKey(ByteBuffer wrappedSecretKey) {
if (kmsClient == null) {
throw new IllegalStateException("Null KmsClient. UnwrapKey can't be called from workers");
}

return kmsClient.unwrapKey(wrappedSecretKey, tableKeyId);
}
}