Skip to content
Closed
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
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 plainOutputFile() {
throw new UnsupportedOperationException("Not implemented");
}
}
Original file line number Diff line number Diff line change
Expand Up @@ -39,6 +39,11 @@ public ByteBuffer buffer() {
public EncryptionKeyMetadata copy() {
return this;
}

@Override
public ByteBuffer encryptionKey() {
return null;
}
};

static EncryptionKeyMetadata empty() {
Expand All @@ -49,4 +54,12 @@ static EncryptionKeyMetadata empty() {
ByteBuffer buffer();

EncryptionKeyMetadata copy();

default ByteBuffer encryptionKey() {
throw new UnsupportedOperationException("Not implemented");
}

default ByteBuffer aadPrefix() {
throw new UnsupportedOperationException("Not implemented");
}
}
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) {
return new BaseEncryptedOutputFile(encryptingOutputFile, EncryptionKeyMetadata.empty());
}

private EncryptedFiles() {}
}
107 changes: 107 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,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<String, String> 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<KeyManagementClient> 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<String, String> 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);
}
}
132 changes: 0 additions & 132 deletions core/src/main/java/org/apache/iceberg/encryption/KeyMetadata.java

This file was deleted.

11 changes: 11 additions & 0 deletions core/src/main/java/org/apache/iceberg/io/FileAppenderFactory.java
Original file line number Diff line number Diff line change
Expand Up @@ -40,6 +40,17 @@ public interface FileAppenderFactory<T> {
*/
FileAppender<T> newAppender(OutputFile outputFile, FileFormat fileFormat);

/**
* Create a new {@link FileAppender}.
*
* @param outputFile an EncryptedOutputFile used to create an output stream.
* @param fileFormat File format.
* @return a newly created {@link FileAppender}
*/
default FileAppender<T> newAppender(EncryptedOutputFile outputFile, FileFormat fileFormat) {
return newAppender(outputFile.encryptingOutputFile(), fileFormat);
}

/**
* Create a new {@link DataWriter}.
*
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -118,7 +118,7 @@ public DataWriter<T> newDataWriter(

case PARQUET:
Parquet.DataWriteBuilder parquetBuilder =
Parquet.writeData(outputFile)
Parquet.writeData(file)
Copy link
Contributor

@rdblue rdblue Dec 11, 2023

Choose a reason for hiding this comment

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

Since encryptingOutputFile will create an AesGcmOutputFile, I don't think it should be called unless it is going to be used. I think outputFile should be removed and the branches that pass an OutputFile (ORC and Avro) should call file.encryptingOutputFile() inline.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

SGTM

.schema(dataSchema)
.setAll(properties)
.metricsConfig(metricsConfig)
Expand Down Expand Up @@ -186,7 +186,7 @@ public EqualityDeleteWriter<T> newEqualityDeleteWriter(

case PARQUET:
Parquet.DeleteWriteBuilder parquetBuilder =
Parquet.writeDeletes(outputFile)
Parquet.writeDeletes(file)
.setAll(properties)
.metricsConfig(metricsConfig)
.rowSchema(equalityDeleteRowSchema)
Expand Down Expand Up @@ -254,7 +254,7 @@ public PositionDeleteWriter<T> newPositionDeleteWriter(

case PARQUET:
Parquet.DeleteWriteBuilder parquetBuilder =
Parquet.writeDeletes(outputFile)
Parquet.writeDeletes(file)
.setAll(properties)
.metricsConfig(metricsConfig)
.rowSchema(positionDeleteRowSchema)
Expand Down
9 changes: 9 additions & 0 deletions data/src/main/java/org/apache/iceberg/data/DeleteFilter.java
Original file line number Diff line number Diff line change
Expand Up @@ -36,6 +36,8 @@
import org.apache.iceberg.deletes.DeleteCounter;
import org.apache.iceberg.deletes.Deletes;
import org.apache.iceberg.deletes.PositionDeleteIndex;
import org.apache.iceberg.encryption.EncryptionKeyMetadata;
import org.apache.iceberg.encryption.EncryptionUtil;
import org.apache.iceberg.expressions.Expressions;
import org.apache.iceberg.io.CloseableIterable;
import org.apache.iceberg.io.InputFile;
Expand Down Expand Up @@ -293,6 +295,13 @@ private CloseableIterable<Record> openDeletes(DeleteFile deleteFile, Schema dele
builder.filter(Expressions.equal(MetadataColumns.DELETE_FILE_PATH.name(), filePath));
}

if (deleteFile.keyMetadata() != null) {
EncryptionKeyMetadata keyMetadata =
EncryptionUtil.parseKeyMetadata(deleteFile.keyMetadata());
Copy link
Contributor

Choose a reason for hiding this comment

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

Cast or parse would be better here, too.

Copy link
Contributor Author

Choose a reason for hiding this comment

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

keyMetadata() is always a ByteBuffer here, so it has to be parsed.

Copy link
Contributor

Choose a reason for hiding this comment

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

Thanks for clarifying. That makes sense.

builder.withFileEncryptionKey(keyMetadata.encryptionKey());
builder.withAADPrefix(keyMetadata.aadPrefix());
}

return builder.build();

case ORC:
Expand Down
Loading