diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index 3e988eed4a1f..328a8cd7c32c 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -742,6 +742,12 @@ test + + software.amazon.awssdk + dynamodb + test + + software.amazon.awssdk diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CommitTaskData.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CommitTaskData.java index 48838f09c998..e8566e12f04c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CommitTaskData.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/CommitTaskData.java @@ -33,6 +33,7 @@ public record CommitTaskData( Optional referencedDataFile, Optional> fileSplitOffsets, int sortOrderId, + Optional encryptionKeyMetadata, Optional serializedDeletionVector) { public CommitTaskData @@ -46,6 +47,7 @@ public record CommitTaskData( requireNonNull(referencedDataFile, "referencedDataFile is null"); requireNonNull(fileSplitOffsets, "fileSplitOffsets is null"); checkArgument(content == FileContent.DATA || sortOrderId == SortOrder.unsorted().orderId(), "Sorted order id can be present only for data files"); + requireNonNull(encryptionKeyMetadata, "encryptionKeyMetadata is null"); requireNonNull(serializedDeletionVector, "serializedDeletionVector is null"); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java index 59cdb6fac9b6..5460f50d30a4 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergConfig.java @@ -104,6 +104,8 @@ public class IcebergConfig private int metadataParallelism = 8; private boolean bucketExecutionEnabled = true; private boolean fileBasedConflictDetectionEnabled = true; + private Optional encryptionKmsType = Optional.empty(); + private List encryptionKmsProperties = ImmutableList.of(); public CatalogType getCatalogType() { @@ -695,4 +697,50 @@ public IcebergConfig setFileBasedConflictDetectionEnabled(boolean fileBasedConfl this.fileBasedConflictDetectionEnabled = fileBasedConflictDetectionEnabled; return this; } + + public Optional getEncryptionKmsType() + { + return encryptionKmsType; + } + + @Config("iceberg.encryption.kms-type") + @ConfigDescription("KMS type for Iceberg table encryption") + public IcebergConfig setEncryptionKmsType(EncryptionKmsType encryptionKmsType) + { + this.encryptionKmsType = Optional.ofNullable(encryptionKmsType); + return this; + } + + public List getEncryptionKmsProperties() + { + return encryptionKmsProperties; + } + + @Config("iceberg.encryption.kms-properties") + @ConfigDescription("Catalog-level KMS client properties in key=value format") + public IcebergConfig setEncryptionKmsProperties(List encryptionKmsProperties) + { + this.encryptionKmsProperties = Optional.ofNullable(encryptionKmsProperties) + .map(ImmutableList::copyOf) + .orElseGet(ImmutableList::of); + return this; + } + + public enum EncryptionKmsType + { + AWS("org.apache.iceberg.aws.AwsKeyManagementClient"), + GCP("org.apache.iceberg.gcp.GcpKeyManagementClient"); + + private final String kmsClientClassName; + + EncryptionKmsType(String kmsClientClassName) + { + this.kmsClientClassName = kmsClientClassName; + } + + public String getKmsClientClassName() + { + return kmsClientClassName; + } + } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriter.java index f6616bab1c9e..a39fe4dfee3d 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriter.java @@ -24,5 +24,10 @@ public interface IcebergFileWriter { FileMetrics getFileMetrics(); + default Optional getEncryptionKeyMetadata() + { + return Optional.empty(); + } + record FileMetrics(Metrics metrics, Optional> splitOffsets) {} } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java index 332b3117ecdc..7d5b8bc1b54b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergFileWriterFactory.java @@ -33,15 +33,26 @@ import io.trino.plugin.hive.HiveCompressionCodec; import io.trino.plugin.hive.HiveCompressionOption; import io.trino.plugin.hive.orc.OrcWriterConfig; +import io.trino.plugin.iceberg.fileio.EncryptedTrinoInputFile; +import io.trino.plugin.iceberg.fileio.EncryptedTrinoOutputFile; +import io.trino.plugin.iceberg.fileio.ForwardingInputFile; import io.trino.plugin.iceberg.fileio.ForwardingOutputFile; import io.trino.spi.NodeVersion; +import io.trino.spi.Page; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.type.Type; import io.trino.spi.type.TypeManager; import org.apache.iceberg.MetricsConfig; import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.EncryptionUtil; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; import org.weakref.jmx.Managed; import java.io.Closeable; @@ -133,13 +144,14 @@ public IcebergFileWriter createDataFileWriter( ConnectorSession session, IcebergFileFormat fileFormat, MetricsConfig metricsConfig, - Map storageProperties) + Map storageProperties, + Optional encryptionManager) { return switch (fileFormat) { // TODO use metricsConfig https://github.com/trinodb/trino/issues/9791 - case PARQUET -> createParquetWriter(MetricsConfig.getDefault(), fileSystem, outputPath, icebergSchema, session, storageProperties); - case ORC -> createOrcWriter(metricsConfig, fileSystem, outputPath, icebergSchema, session, storageProperties, getOrcStringStatisticsLimit(session)); - case AVRO -> createAvroWriter(fileSystem, outputPath, icebergSchema, storageProperties); + case PARQUET -> createParquetWriter(MetricsConfig.getDefault(), fileSystem, outputPath, icebergSchema, session, storageProperties, encryptionManager); + case ORC -> createOrcWriter(metricsConfig, fileSystem, outputPath, icebergSchema, session, storageProperties, getOrcStringStatisticsLimit(session), encryptionManager); + case AVRO -> createAvroWriter(fileSystem, outputPath, icebergSchema, storageProperties, encryptionManager); }; } @@ -148,12 +160,13 @@ public IcebergFileWriter createPositionDeleteWriter( Location outputPath, ConnectorSession session, IcebergFileFormat fileFormat, - Map storageProperties) + Map storageProperties, + Optional encryptionManager) { return switch (fileFormat) { - case PARQUET -> createParquetWriter(FULL_METRICS_CONFIG, fileSystem, outputPath, POSITION_DELETE_SCHEMA, session, storageProperties); - case ORC -> createOrcWriter(FULL_METRICS_CONFIG, fileSystem, outputPath, POSITION_DELETE_SCHEMA, session, storageProperties, DataSize.ofBytes(Integer.MAX_VALUE)); - case AVRO -> createAvroWriter(fileSystem, outputPath, POSITION_DELETE_SCHEMA, storageProperties); + case PARQUET -> createParquetWriter(FULL_METRICS_CONFIG, fileSystem, outputPath, POSITION_DELETE_SCHEMA, session, storageProperties, encryptionManager); + case ORC -> createOrcWriter(FULL_METRICS_CONFIG, fileSystem, outputPath, POSITION_DELETE_SCHEMA, session, storageProperties, DataSize.ofBytes(Integer.MAX_VALUE), encryptionManager); + case AVRO -> createAvroWriter(fileSystem, outputPath, POSITION_DELETE_SCHEMA, storageProperties, encryptionManager); }; } @@ -163,7 +176,8 @@ private IcebergFileWriter createParquetWriter( Location outputPath, Schema icebergSchema, ConnectorSession session, - Map storageProperties) + Map storageProperties, + Optional encryptionManager) { List fileColumnNames = icebergSchema.columns().stream() .map(Types.NestedField::name) @@ -173,7 +187,8 @@ private IcebergFileWriter createParquetWriter( .collect(toImmutableList()); try { - TrinoOutputFile outputFile = fileSystem.newOutputFile(outputPath); + EncryptedOutput encryptedOutput = createOutputFile(fileSystem, outputPath, encryptionManager); + TrinoOutputFile outputFile = encryptedOutput.trinoOutputFile(); Closeable rollbackAction = () -> fileSystem.deleteFile(outputPath); @@ -188,7 +203,7 @@ private IcebergFileWriter createParquetWriter( HiveCompressionCodec compressionCodec = getHiveCompressionCodec(PARQUET, storageProperties) .orElse(toCompressionCodec(hiveCompressionOption)); - return new IcebergParquetFileWriter( + IcebergFileWriter writer = new IcebergParquetFileWriter( metricsConfig, outputFile, rollbackAction, @@ -201,6 +216,7 @@ private IcebergFileWriter createParquetWriter( compressionCodec.getParquetCompressionCodec() .orElseThrow(() -> new TrinoException(NOT_SUPPORTED, "Compression codec %s not supported for Parquet".formatted(compressionCodec))), nodeVersion.toString()); + return withEncryptionKeyMetadata(writer, encryptedOutput.keyMetadata()); } catch (IOException | UncheckedIOException e) { throw new TrinoException(ICEBERG_WRITER_OPEN_ERROR, "Error creating Parquet file", e); @@ -214,10 +230,12 @@ private IcebergFileWriter createOrcWriter( Schema icebergSchema, ConnectorSession session, Map storageProperties, - DataSize stringStatisticsLimit) + DataSize stringStatisticsLimit, + Optional encryptionManager) { try { - OrcDataSink orcDataSink = OutputStreamOrcDataSink.create(fileSystem.newOutputFile(outputPath)); + EncryptedOutput encryptedOutput = createOutputFile(fileSystem, outputPath, encryptionManager); + OrcDataSink orcDataSink = OutputStreamOrcDataSink.create(encryptedOutput.trinoOutputFile()); Closeable rollbackAction = () -> fileSystem.deleteFile(outputPath); @@ -234,7 +252,7 @@ private IcebergFileWriter createOrcWriter( if (isOrcWriterValidate(session)) { validationInputFactory = Optional.of(() -> { try { - TrinoInputFile inputFile = fileSystem.newInputFile(outputPath); + TrinoInputFile inputFile = createValidationInputFile(fileSystem, outputPath, encryptedOutput.keyMetadata(), encryptionManager); return new TrinoOrcDataSource(inputFile, new OrcReaderOptions(), readStats); } catch (IOException | UncheckedIOException e) { @@ -246,7 +264,7 @@ private IcebergFileWriter createOrcWriter( HiveCompressionCodec compressionCodec = getHiveCompressionCodec(ORC, storageProperties) .orElse(toCompressionCodec(hiveCompressionOption)); - return new IcebergOrcFileWriter( + IcebergFileWriter writer = new IcebergOrcFileWriter( metricsConfig, icebergSchema, orcDataSink, @@ -270,6 +288,7 @@ private IcebergFileWriter createOrcWriter( validationInputFactory, getOrcWriterValidateMode(session), orcWriterStats); + return withEncryptionKeyMetadata(writer, encryptedOutput.keyMetadata()); } catch (IOException | UncheckedIOException e) { throw new TrinoException(ICEBERG_WRITER_OPEN_ERROR, "Error creating ORC file", e); @@ -299,7 +318,8 @@ private IcebergFileWriter createAvroWriter( TrinoFileSystem fileSystem, Location outputPath, Schema icebergSchema, - Map storageProperties) + Map storageProperties, + Optional encryptionManager) { Closeable rollbackAction = () -> fileSystem.deleteFile(outputPath); @@ -310,11 +330,113 @@ private IcebergFileWriter createAvroWriter( HiveCompressionCodec compressionCodec = getHiveCompressionCodec(AVRO, storageProperties) .orElse(toCompressionCodec(hiveCompressionOption)); - return new IcebergAvroFileWriter( - new ForwardingOutputFile(fileSystem, outputPath), + EncryptedOutput encryptedOutput = createOutputFile(fileSystem, outputPath, encryptionManager); + + IcebergFileWriter writer = new IcebergAvroFileWriter( + encryptedOutput.icebergOutputFile(), rollbackAction, icebergSchema, columnTypes, compressionCodec); + return withEncryptionKeyMetadata(writer, encryptedOutput.keyMetadata()); + } + + private static TrinoInputFile createValidationInputFile( + TrinoFileSystem fileSystem, + Location outputPath, + Optional keyMetadata, + Optional encryptionManager) + { + TrinoInputFile inputFile = fileSystem.newInputFile(outputPath); + if (keyMetadata.isEmpty() || encryptionManager.isEmpty()) { + return inputFile; + } + InputFile encryptedInputFile = new ForwardingInputFile(inputFile); + InputFile decryptedInputFile = encryptionManager.get().decrypt(EncryptedFiles.encryptedInput(encryptedInputFile, keyMetadata.get())); + return new EncryptedTrinoInputFile(inputFile, decryptedInputFile); + } + + private EncryptedOutput createOutputFile(TrinoFileSystem fileSystem, Location outputPath, Optional encryptionManager) + { + OutputFile icebergOutputFile = new ForwardingOutputFile(fileSystem, outputPath); + EncryptedOutputFile encryptedOutputFile = encryptionManager + .map(manager -> manager.encrypt(icebergOutputFile)) + .orElseGet(() -> EncryptionUtil.plainAsEncryptedOutput(icebergOutputFile)); + OutputFile encryptingOutputFile = encryptedOutputFile.encryptingOutputFile(); + TrinoOutputFile trinoOutputFile = new EncryptedTrinoOutputFile(outputPath, encryptingOutputFile); + Optional keyMetadata = Optional.ofNullable(encryptedOutputFile.keyMetadata().buffer()) + .map(ByteBuffers::toByteArray); + return new EncryptedOutput(trinoOutputFile, encryptingOutputFile, keyMetadata); + } + + private static IcebergFileWriter withEncryptionKeyMetadata(IcebergFileWriter writer, Optional keyMetadata) + { + if (keyMetadata.isEmpty()) { + return writer; + } + return new EncryptionMetadataFileWriter(writer, keyMetadata); + } + + private record EncryptedOutput(TrinoOutputFile trinoOutputFile, OutputFile icebergOutputFile, Optional keyMetadata) {} + + private static class EncryptionMetadataFileWriter + implements IcebergFileWriter + { + private final IcebergFileWriter delegate; + private final Optional keyMetadata; + + private EncryptionMetadataFileWriter(IcebergFileWriter delegate, Optional keyMetadata) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + this.keyMetadata = requireNonNull(keyMetadata, "keyMetadata is null"); + } + + @Override + public FileMetrics getFileMetrics() + { + return delegate.getFileMetrics(); + } + + @Override + public Optional getEncryptionKeyMetadata() + { + return keyMetadata; + } + + @Override + public long getWrittenBytes() + { + return delegate.getWrittenBytes(); + } + + @Override + public long getMemoryUsage() + { + return delegate.getMemoryUsage(); + } + + @Override + public void appendRows(Page dataPage) + { + delegate.appendRows(dataPage); + } + + @Override + public Closeable commit() + { + return delegate.commit(); + } + + @Override + public void rollback() + { + delegate.rollback(); + } + + @Override + public long getValidationCpuNanos() + { + return delegate.getValidationCpuNanos(); + } } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeSink.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeSink.java index e6549444600a..639285de0c36 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeSink.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMergeSink.java @@ -35,6 +35,7 @@ import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.types.Type; @@ -66,6 +67,7 @@ public class IcebergMergeSink private final ConnectorSession session; private final IcebergFileFormat fileFormat; private final Map storageProperties; + private final Optional encryptionManager; private final Schema schema; private final Map partitionsSpecs; private final ConnectorPageSink insertPageSink; @@ -82,6 +84,7 @@ public IcebergMergeSink( ConnectorSession session, IcebergFileFormat fileFormat, Map storageProperties, + Optional encryptionManager, Schema schema, Map partitionsSpecs, ConnectorPageSink insertPageSink, @@ -95,6 +98,7 @@ public IcebergMergeSink( this.session = requireNonNull(session, "session is null"); this.fileFormat = requireNonNull(fileFormat, "fileFormat is null"); this.storageProperties = ImmutableMap.copyOf(requireNonNull(storageProperties, "storageProperties is null")); + this.encryptionManager = requireNonNull(encryptionManager, "encryptionManager is null"); this.schema = requireNonNull(schema, "schema is null"); this.partitionsSpecs = ImmutableMap.copyOf(requireNonNull(partitionsSpecs, "partitionsSpecs is null")); this.insertPageSink = requireNonNull(insertPageSink, "insertPageSink is null"); @@ -181,6 +185,7 @@ else if (formatVersion == 3) { Optional.of(dataFilePath.toStringUtf8()), Optional.empty(), // unused for v3 SortOrder.unsorted().orderId(), + Optional.empty(), Optional.of(deletionVector.serialize().getBytes())); fragments.add(wrappedBuffer(jsonCodec.toJsonBytes(task))); })); @@ -209,7 +214,8 @@ private PositionDeleteWriter createPositionDeleteWriter(String dataFilePath, Par fileSystem, session, fileFormat, - storageProperties); + storageProperties, + encryptionManager); } private Slice writePositionDeletes(PositionDeleteWriter writer, DeletionVector rowsToDelete) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java index fe94a28f4dcd..68763c4dfc4a 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergMetadata.java @@ -232,6 +232,7 @@ import java.io.IOException; import java.io.UncheckedIOException; +import java.nio.ByteBuffer; import java.time.Instant; import java.time.LocalDate; import java.time.LocalDateTime; @@ -347,6 +348,8 @@ import static io.trino.plugin.iceberg.IcebergTableProperties.COMPRESSION_CODEC; import static io.trino.plugin.iceberg.IcebergTableProperties.DATA_LOCATION_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.DELETE_AFTER_COMMIT_ENABLED; +import static io.trino.plugin.iceberg.IcebergTableProperties.ENCRYPTION_DATA_KEY_LENGTH_PROPERTY; +import static io.trino.plugin.iceberg.IcebergTableProperties.ENCRYPTION_KEY_ID_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.EXTRA_PROPERTIES_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.FILE_FORMAT_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.FORMAT_VERSION_PROPERTY; @@ -467,6 +470,8 @@ import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL; import static org.apache.iceberg.TableProperties.DELETE_ISOLATION_LEVEL_DEFAULT; +import static org.apache.iceberg.TableProperties.ENCRYPTION_DEK_LENGTH; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES; import static org.apache.iceberg.TableProperties.MANIFEST_TARGET_SIZE_BYTES_DEFAULT; @@ -509,6 +514,8 @@ public class IcebergMetadata .add(PARQUET_BLOOM_FILTER_COLUMNS_PROPERTY) .add(PARTITIONING_PROPERTY) .add(SORTED_BY_PROPERTY) + .add(ENCRYPTION_KEY_ID_PROPERTY) + .add(ENCRYPTION_DATA_KEY_LENGTH_PROPERTY) .build(); private static final String SYSTEM_SCHEMA = "system"; @@ -610,7 +617,7 @@ private Optional getOrLoadTableCredentials(ConnectorS schemaTableName, () -> { BaseTable baseTable = catalog.loadTable(session, schemaTableName); - return new IcebergTableCredentials(baseTable.io().properties()); + return new IcebergTableCredentials(getFileIoProperties(baseTable)); })); } catch (UncheckedExecutionException e) { @@ -770,11 +777,6 @@ private static void validateTableForTrino(BaseTable table, OptionalLong tableSna if (metadata.formatVersion() < 3) { return; } - - // Reject Iceberg table encryption - if (!metadata.encryptionKeys().isEmpty() || snapshot.keyId() != null || metadata.properties().containsKey("encryption.key-id")) { - throw new TrinoException(NOT_SUPPORTED, "Iceberg table encryption is not supported"); - } } private IcebergTableHandle tableHandleForCurrentSnapshot(ConnectorSession session, SchemaTableName tableName, BaseTable table) @@ -1498,7 +1500,7 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con try { // S3 Tables internally assigns a unique location for each table if (!isS3Tables(location.toString())) { - TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), transaction.table().io().properties()); + TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), getFileIoProperties(transaction.table())); if (!replace && fileSystem.listFiles(location).hasNext()) { throw new TrinoException(ICEBERG_FILESYSTEM_ERROR, format("" + "Cannot create a table on a non-empty location: %s, set 'iceberg.unique-table-location=true' in your Iceberg catalog properties " + @@ -1645,6 +1647,11 @@ private IcebergWritableTableHandle newWritableTableHandle(SchemaTableName name, table.properties()); } + private static Map getFileIoProperties(Table table) + { + return IcebergUtil.getFileIoProperties(table); + } + private static SortFieldInfo getSupportedSortFields(Schema schema, SortOrder sortOrder) { if (!sortOrder.isSorted()) { @@ -1713,6 +1720,7 @@ public Optional finishInsert( .withMetrics(task.metrics().metrics()) .withSortOrder(sortOrders.get(task.sortOrderId())); task.fileSplitOffsets().ifPresent(builder::withSplitOffsets); + task.encryptionKeyMetadata().ifPresent(metadata -> builder.withEncryptionKeyMetadata(ByteBuffer.wrap(metadata))); if (!icebergTable.spec().fields().isEmpty()) { String partitionDataJson = task.partitionDataJson() @@ -2161,6 +2169,7 @@ private Map finishOptimize(ConnectorSession session, IcebergTableE .withMetrics(task.metrics().metrics()) .withSortOrder(sortOrders.get(task.sortOrderId())); task.fileSplitOffsets().ifPresent(builder::withSplitOffsets); + task.encryptionKeyMetadata().ifPresent(metadata -> builder.withEncryptionKeyMetadata(ByteBuffer.wrap(metadata))); if (!icebergTable.spec().fields().isEmpty()) { String partitionDataJson = task.partitionDataJson() @@ -2557,7 +2566,7 @@ public Map executeRemoveOrphanFiles(ConnectorSession session, Iceb } Instant expiration = session.getStart().minusMillis(retention.toMillis()); - return removeOrphanFiles(table, session, executeHandle.schemaTableName(), expiration, table.io().properties()); + return removeOrphanFiles(table, session, executeHandle.schemaTableName(), expiration, getFileIoProperties(table)); } private Map removeOrphanFiles(Table table, ConnectorSession session, SchemaTableName schemaTableName, Instant expiration, Map fileIoProperties) @@ -2641,7 +2650,7 @@ public Map executeAddFiles(ConnectorSession session, IcebergTableE { IcebergAddFilesHandle addFilesHandle = (IcebergAddFilesHandle) executeHandle.procedureHandle(); Table table = catalog.loadTable(session, executeHandle.schemaTableName()); - TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), table.io().properties()); + TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), getFileIoProperties(table)); long addedDataFiles = addFiles( session, fileSystem, @@ -2658,7 +2667,7 @@ public Map executeAddFilesFromTable(ConnectorSession session, Iceb { IcebergAddFilesFromTableHandle addFilesHandle = (IcebergAddFilesFromTableHandle) executeHandle.procedureHandle(); Table table = catalog.loadTable(session, executeHandle.schemaTableName()); - TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), table.io().properties()); + TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), getFileIoProperties(table)); long addedDataFiles = addFilesFromTable( session, fileSystem, @@ -2905,6 +2914,32 @@ public void setTableProperties(ConnectorSession session, ConnectorTableHandle ta updateProperties.set(WRITE_DATA_LOCATION, dataLocation); } + String existingEncryptionKeyId = icebergTable.properties().get(ENCRYPTION_TABLE_KEY); + if (properties.containsKey(ENCRYPTION_KEY_ID_PROPERTY)) { + String newEncryptionKeyId = (String) properties.get(ENCRYPTION_KEY_ID_PROPERTY) + .orElseThrow(() -> new IllegalArgumentException("The encryption_key_id property cannot be empty")); + if (existingEncryptionKeyId != null && !existingEncryptionKeyId.equals(newEncryptionKeyId)) { + throw new TrinoException(NOT_SUPPORTED, "The encryption_key_id property cannot be modified once set"); + } + if (existingEncryptionKeyId == null) { + updateProperties.set(ENCRYPTION_TABLE_KEY, newEncryptionKeyId); + existingEncryptionKeyId = newEncryptionKeyId; + } + } + + if (properties.containsKey(ENCRYPTION_DATA_KEY_LENGTH_PROPERTY)) { + int dataKeyLength = (int) properties.get(ENCRYPTION_DATA_KEY_LENGTH_PROPERTY) + .orElseThrow(() -> new IllegalArgumentException("The encryption_data_key_length property cannot be empty")); + if (existingEncryptionKeyId == null && !properties.containsKey(ENCRYPTION_KEY_ID_PROPERTY)) { + throw new TrinoException(INVALID_TABLE_PROPERTY, "encryption_data_key_length requires encryption_key_id"); + } + String existingLength = icebergTable.properties().get(ENCRYPTION_DEK_LENGTH); + if (existingEncryptionKeyId != null && existingLength != null && Integer.parseInt(existingLength) != dataKeyLength) { + throw new TrinoException(NOT_SUPPORTED, "The encryption_data_key_length property cannot be modified once set"); + } + updateProperties.set(ENCRYPTION_DEK_LENGTH, Integer.toString(dataKeyLength)); + } + try { updateProperties.commit(); } @@ -3650,6 +3685,7 @@ private void finishWrite(ConnectorSession session, IcebergTableHandle table, Col .withMetrics(task.metrics().metrics()) .withSortOrder(sortOrders.get(task.sortOrderId())); task.fileSplitOffsets().ifPresent(builder::withSplitOffsets); + task.encryptionKeyMetadata().ifPresent(metadata -> builder.withEncryptionKeyMetadata(ByteBuffer.wrap(metadata))); if (!icebergTable.spec().fields().isEmpty()) { String partitionDataJson = task.partitionDataJson() @@ -3683,6 +3719,7 @@ private void finishWrite(ConnectorSession session, IcebergTableHandle table, Col .withFileSizeInBytes(task.fileSizeInBytes()) .withMetrics(task.metrics().metrics()); task.fileSplitOffsets().ifPresent(deleteBuilder::withSplitOffsets); + task.encryptionKeyMetadata().ifPresent(metadata -> deleteBuilder.withEncryptionKeyMetadata(ByteBuffer.wrap(metadata))); toPartitionData(partitionSpec, schema, task.partitionDataJson()).ifPresent(deleteBuilder::withPartition); rowDelta.addDeletes(deleteBuilder.build()); } @@ -4342,6 +4379,7 @@ public Optional finishRefreshMaterializedView( .withMetrics(task.metrics().metrics()) .withSortOrder(sortOrders.get(task.sortOrderId())); task.fileSplitOffsets().ifPresent(builder::withSplitOffsets); + task.encryptionKeyMetadata().ifPresent(metadata -> builder.withEncryptionKeyMetadata(ByteBuffer.wrap(metadata))); if (!icebergTable.spec().fields().isEmpty()) { String partitionDataJson = task.partitionDataJson() diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java index b39e3c62cd9c..3a97e0ac1b04 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergModule.java @@ -36,6 +36,7 @@ import io.trino.plugin.iceberg.cache.IcebergCacheKeyProvider; import io.trino.plugin.iceberg.delete.DefaultDeletionVectorWriter; import io.trino.plugin.iceberg.delete.DeletionVectorWriter; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.plugin.iceberg.functions.IcebergFunctionProvider; import io.trino.plugin.iceberg.functions.tablechanges.TableChangesFunctionProcessorProviderFactory; @@ -104,6 +105,7 @@ public void configure(Binder binder) binder.bind(TableStatisticsReader.class).in(Scopes.SINGLETON); binder.bind(TableStatisticsWriter.class).in(Scopes.SINGLETON); binder.bind(DeletionVectorWriter.class).to(DefaultDeletionVectorWriter.class).in(Scopes.SINGLETON); + binder.bind(IcebergEncryptionManagerFactory.class).in(Scopes.SINGLETON); binder.bind(IcebergMetadataFactory.class).in(Scopes.SINGLETON); newOptionalBinder(binder, Key.get(HiveMetastoreFactory.class, RawHiveMetastoreFactory.class)); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java index 8b51e6f3b3a1..0bc3d8133b5f 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSink.java @@ -42,6 +42,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; +import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.transforms.Transform; import org.apache.iceberg.types.TypeUtil; @@ -115,6 +116,7 @@ public class IcebergPageSink private final long targetMaxFileSize; private final long idleWriterMinFileSize; private final Map storageProperties; + private final Optional encryptionManager; private final List sortFields; private final int sortOrderId; private final boolean sortedWritingEnabled; @@ -149,6 +151,7 @@ public IcebergPageSink( ConnectorSession session, IcebergFileFormat fileFormat, Map storageProperties, + Optional encryptionManager, int maxOpenWriters, List sortFields, int sortOrderId, @@ -173,6 +176,7 @@ public IcebergPageSink( this.targetMaxFileSize = IcebergSessionProperties.getTargetMaxFileSize(session); this.idleWriterMinFileSize = IcebergSessionProperties.getIdleWriterMinFileSize(session); this.storageProperties = requireNonNull(storageProperties, "storageProperties is null"); + this.encryptionManager = requireNonNull(encryptionManager, "encryptionManager is null"); this.sortFields = requireNonNull(sortFields, "sortFields is null"); this.sortedWritingEnabled = isSortedWritingEnabled(session); this.sortingFileWriterBufferSize = requireNonNull(sortingFileWriterBufferSize, "sortingFileWriterBufferSize is null"); @@ -424,10 +428,12 @@ private void closeWriter(int writerIndex) writers.set(writerIndex, null); currentOpenWriters--; + long fileSizeInBytes = writer.getWrittenBytes(); + Optional encryptionKeyMetadata = encryptionKeyMetadata(writer); CommitTaskData task = new CommitTaskData( writeContext.getPath(), fileFormat, - writer.getWrittenBytes(), + fileSizeInBytes, new MetricsWrapper(writer.getFileMetrics().metrics()), PartitionSpecParser.toJson(partitionSpec), writeContext.getPartitionData().map(PartitionData::toJson), @@ -435,6 +441,7 @@ private void closeWriter(int writerIndex) Optional.empty(), writer.getFileMetrics().splitOffsets(), sortOrderId, + encryptionKeyMetadata, Optional.empty()); commitTasks.add(wrappedBuffer(jsonCodec.toJsonBytes(task))); @@ -449,11 +456,17 @@ private WriteContext createWriter(String outputPath, Optional par session, fileFormat, metricsConfig, - storageProperties); + storageProperties, + encryptionManager); return new WriteContext(writer, outputPath, partitionData); } + private static Optional encryptionKeyMetadata(IcebergFileWriter writer) + { + return writer.getEncryptionKeyMetadata(); + } + private Optional getPartitionData(List columns, Page page, int position) { if (columns.isEmpty()) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java index fcafc808ec00..6672b14c377b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSinkProvider.java @@ -17,6 +17,7 @@ import io.airlift.json.JsonCodec; import io.airlift.units.DataSize; import io.trino.plugin.hive.SortingFileWriterConfig; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.procedure.IcebergOptimizeHandle; import io.trino.plugin.iceberg.procedure.IcebergTableExecuteHandle; import io.trino.spi.PageIndexerFactory; @@ -38,6 +39,8 @@ import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.types.Types; @@ -64,6 +67,7 @@ public class IcebergPageSinkProvider private final Optional sortingFileWriterLocalStagingPath; private final TypeManager typeManager; private final PageSorter pageSorter; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; @Inject public IcebergPageSinkProvider( @@ -74,7 +78,8 @@ public IcebergPageSinkProvider( SortingFileWriterConfig sortingFileWriterConfig, IcebergConfig icebergConfig, TypeManager typeManager, - PageSorter pageSorter) + PageSorter pageSorter, + IcebergEncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.jsonCodec = requireNonNull(jsonCodec, "jsonCodec is null"); @@ -85,6 +90,7 @@ public IcebergPageSinkProvider( this.sortingFileWriterLocalStagingPath = icebergConfig.getSortedWritingLocalStagingPath(); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.pageSorter = requireNonNull(pageSorter, "pageSorter is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override @@ -112,6 +118,7 @@ private IcebergPageSink createPageSink(ConnectorSession session, IcebergWritable String partitionSpecJson = tableHandle.partitionsSpecsAsJson().get(tableHandle.partitionSpecId()); PartitionSpec partitionSpec = PartitionSpecParser.fromJson(schema, partitionSpecJson); LocationProvider locationProvider = getLocationProvider(tableHandle.name(), tableHandle.outputPath(), tableHandle.storageProperties()); + Optional encryptionManager = encryptionManager(tableHandle.storageProperties()); return new IcebergPageSink( schema, partitionSpec, @@ -124,6 +131,7 @@ private IcebergPageSink createPageSink(ConnectorSession session, IcebergWritable session, tableHandle.fileFormat(), tableHandle.storageProperties(), + encryptionManager, maxPartitionsPerWriter(session), tableHandle.sortFields(), tableHandle.sortOrderId(), @@ -145,6 +153,7 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa PartitionSpec partitionSpec = PartitionSpecParser.fromJson(schema, optimizeHandle.partitionSpecAsJson()); LocationProvider locationProvider = getLocationProvider(executeHandle.schemaTableName(), executeHandle.tableLocation(), optimizeHandle.tableStorageProperties()); + Optional encryptionManager = encryptionManager(optimizeHandle.tableStorageProperties()); return new IcebergPageSink( schema, partitionSpec, @@ -157,6 +166,7 @@ public ConnectorPageSink createPageSink(ConnectorTransactionHandle transactionHa session, optimizeHandle.fileFormat(), optimizeHandle.tableStorageProperties(), + encryptionManager, maxPartitionsPerWriter(session), optimizeHandle.sortFields(), optimizeHandle.sortOrderId(), @@ -200,6 +210,7 @@ public ConnectorMergeSink createMergeSink(ConnectorTransactionHandle transaction outputSchema = SchemaParser.fromJson(tableHandle.schemaAsJson()); } ConnectorPageSink pageSink = createPageSink(session, tableHandle, outputSchema, fileIoProperties); + Optional encryptionManager = encryptionManager(tableHandle.storageProperties()); return new IcebergMergeSink( formatVersion, @@ -210,9 +221,19 @@ public ConnectorMergeSink createMergeSink(ConnectorTransactionHandle transaction session, tableHandle.fileFormat(), tableHandle.storageProperties(), + encryptionManager, schema, partitionsSpecs, pageSink, schema.columns().size()); } + + private Optional encryptionManager(Map tableProperties) + { + EncryptionManager encryptionManager = encryptionManagerFactory.createEncryptionManager(tableProperties); + if (encryptionManager instanceof PlaintextEncryptionManager) { + return Optional.empty(); + } + return Optional.of(encryptionManager); + } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java index a3aeb78de8c0..e92e7cbe93d3 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProvider.java @@ -41,6 +41,8 @@ import io.trino.parquet.ParquetDataSource; import io.trino.parquet.ParquetDataSourceId; import io.trino.parquet.ParquetReaderOptions; +import io.trino.parquet.crypto.DecryptionKeyRetriever; +import io.trino.parquet.crypto.FileDecryptionProperties; import io.trino.parquet.metadata.FileMetadata; import io.trino.parquet.metadata.ParquetMetadata; import io.trino.parquet.predicate.TupleDomainParquetPredicate; @@ -52,10 +54,13 @@ import io.trino.plugin.hive.orc.OrcPageSource; import io.trino.plugin.hive.parquet.ParquetPageSource; import io.trino.plugin.iceberg.IcebergParquetColumnIOConverter.FieldContext; +import io.trino.plugin.iceberg.IcebergSplit.ParquetFileDecryptionData; import io.trino.plugin.iceberg.delete.DeleteFile; import io.trino.plugin.iceberg.delete.DeleteManager; import io.trino.plugin.iceberg.delete.DeletionVector; import io.trino.plugin.iceberg.delete.RowPredicate; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; +import io.trino.plugin.iceberg.fileio.EncryptedTrinoInputFile; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.plugin.iceberg.fileio.ForwardingInputFile; import io.trino.plugin.iceberg.system.files.FilesTablePageSource; @@ -92,12 +97,16 @@ import jakarta.annotation.Nullable; import org.apache.avro.file.DataFileStream; import org.apache.avro.generic.GenericDatumReader; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.MetadataColumns; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.MappedFields; @@ -109,6 +118,7 @@ import org.apache.iceberg.types.Types.NestedField; import org.apache.iceberg.util.StructLikeWrapper; import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.metadata.ColumnPath; import org.apache.parquet.io.MessageColumnIO; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; @@ -227,6 +237,7 @@ public class IcebergPageSourceProvider private final OrcReaderOptions orcReaderOptions; private final ParquetReaderOptions parquetReaderOptions; private final TypeManager typeManager; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; private final DeleteManager unpartitionedTableDeleteManager; private final Map> partitionKeyFactories = new ConcurrentHashMap<>(); private final Map partitionedDeleteManagers = new ConcurrentHashMap<>(); @@ -237,7 +248,8 @@ public IcebergPageSourceProvider( FileFormatDataSourceStats fileFormatDataSourceStats, OrcReaderOptions orcReaderOptions, ParquetReaderOptions parquetReaderOptions, - TypeManager typeManager) + TypeManager typeManager, + IcebergEncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); @@ -245,6 +257,7 @@ public IcebergPageSourceProvider( this.orcReaderOptions = requireNonNull(orcReaderOptions, "orcReaderOptions is null"); this.parquetReaderOptions = requireNonNull(parquetReaderOptions, "parquetReaderOptions is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); this.unpartitionedTableDeleteManager = new DeleteManager(typeManager); } @@ -278,6 +291,7 @@ public ConnectorPageSource createPageSource( org.apache.iceberg.types.Type[] partitionColumnTypes = partitionSpec.fields().stream() .map(field -> field.transform().getResultType(schema.findType(field.sourceId()))) .toArray(org.apache.iceberg.types.Type[]::new); + Optional encryptionManager = encryptionManager(tableHandle.getStorageProperties()); return createPageSource( session, @@ -299,7 +313,10 @@ public ConnectorPageSource createPageSource( getFileIoProperties(connectorTableCredentials), split.getDataSequenceNumber(), split.getFileFirstRowId(), - tableHandle.getNameMappingJson().map(NameMappingParser::fromJson)); + tableHandle.getNameMappingJson().map(NameMappingParser::fromJson), + split.getEncryptionKeyMetadata(), + split.getParquetFileDecryptionData(), + encryptionManager); } public ConnectorPageSource createPageSource( @@ -322,7 +339,10 @@ public ConnectorPageSource createPageSource( Map fileIoProperties, long dataSequenceNumber, OptionalLong fileFirstRowId, - Optional nameMapping) + Optional nameMapping, + Optional encryptionKeyMetadata, + Optional parquetFileDecryptionData, + Optional encryptionManager) { Map> partitionKeys = getPartitionKeys(partitionData, partitionSpec); TupleDomain effectivePredicate = getUnenforcedPredicate( @@ -338,12 +358,17 @@ public ConnectorPageSource createPageSource( // exit early when only reading partition keys from a simple split String partition = partitionSpec.partitionToPath(partitionData); TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), fileIoProperties); - TrinoInputFile inputFile = isUseFileSizeFromMetadata(session) - ? fileSystem.newInputFile(Location.of(path), fileSize) - : fileSystem.newInputFile(Location.of(path)); + TrinoInputFile inputFile = newInputFile(fileSystem, path, fileSize, encryptionKeyMetadata, isUseFileSizeFromMetadata(session)); + boolean useLegacyParquetDecryption = fileFormat == IcebergFileFormat.PARQUET && encryptionKeyMetadata.isPresent(); + Optional effectiveParquetFileDecryptionData = useLegacyParquetDecryption ? Optional.empty() : parquetFileDecryptionData; + + TrinoInputFile decryptedInputFile = inputFile; + if (fileFormat != IcebergFileFormat.PARQUET || useLegacyParquetDecryption) { + decryptedInputFile = decryptInputFileIfNeeded(inputFile, encryptionKeyMetadata, encryptionManager); + } try { if (effectivePredicate.isAll() && - start == 0 && length == inputFile.length() && + start == 0 && length == decryptedInputFile.length() && deletes.isEmpty() && icebergColumns.stream().allMatch(column -> partitionKeys.containsKey(column.getId()))) { return generatePages( @@ -363,12 +388,22 @@ public ConnectorPageSource createPageSource( .filter(not(icebergColumns::contains)) .forEach(requiredColumns::add); + long decryptedFileSize = fileSize; + if ((fileFormat != IcebergFileFormat.PARQUET || useLegacyParquetDecryption) && encryptionKeyMetadata.isPresent()) { + try { + decryptedFileSize = decryptedInputFile.length(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } + ReaderPageSourceWithRowPositions readerPageSourceWithRowPositions = createDataPageSource( session, - inputFile, + decryptedInputFile, start, length, - fileSize, + decryptedFileSize, partitionSpec.specId(), partitionDataJson, fileFormat, @@ -376,6 +411,7 @@ public ConnectorPageSource createPageSource( requiredColumns, effectivePredicate, nameMapping, + effectiveParquetFileDecryptionData, partition, partitionKeys, dataSequenceNumber, @@ -394,8 +430,8 @@ public ConnectorPageSource createPageSource( tableSchema, readerPageSourceWithRowPositions.startRowPosition(), readerPageSourceWithRowPositions.endRowPosition(), - (deleteFile) -> readDeletionVector(fileSystem, deleteFile), - (deleteFile, deleteColumns, tupleDomain) -> openDeleteFile(session, fileSystem, deleteFile, deleteColumns, tupleDomain))); + (deleteFile) -> readDeletionVector(fileSystem, deleteFile, encryptionManager), + (deleteFile, deleteColumns, tupleDomain) -> openDeleteFile(session, fileSystem, deleteFile, deleteColumns, tupleDomain, encryptionManager))); pageSource = TransformConnectorPageSource.create(pageSource, page -> { try { Optional rowPredicate = deletePredicate.get(); @@ -509,11 +545,12 @@ else if (deleteFile.content() == EQUALITY_DELETES) { return requiredColumns.build(); } - private static DeletionVector readDeletionVector(TrinoFileSystem fileSystem, DeleteFile delete) + private static DeletionVector readDeletionVector(TrinoFileSystem fileSystem, DeleteFile delete, Optional encryptionManager) { verify(delete.isDeletionVector(), "Not a deletion vector: %s", delete); - TrinoInputFile trinoInputFile = fileSystem.newInputFile(Location.of(delete.path()), delete.fileSizeInBytes()); - try (TrinoInput trinoInput = trinoInputFile.newInput()) { + TrinoInputFile trinoInputFile = newInputFile(fileSystem, delete.path(), delete.fileSizeInBytes(), delete.encryptionKeyMetadata(), true); + TrinoInputFile inputFile = decryptInputFileIfNeeded(trinoInputFile, delete.encryptionKeyMetadata(), encryptionManager); + try (TrinoInput trinoInput = inputFile.newInput()) { Slice slice = trinoInput.readFully(delete.contentOffset().orElseThrow(), toIntExact(delete.contentSizeInBytes().orElseThrow())); return DeletionVector.builder().deserialize(slice).build().orElseThrow(); } @@ -527,14 +564,35 @@ public ConnectorPageSource openDeleteFile( TrinoFileSystem fileSystem, DeleteFile delete, List columns, - TupleDomain tupleDomain) + TupleDomain tupleDomain, + Optional encryptionManager) { + TrinoInputFile trinoInputFile = newInputFile(fileSystem, delete.path(), delete.fileSizeInBytes(), delete.encryptionKeyMetadata(), true); + Optional parquetFileDecryptionData = delete.parquetFileDecryptionData(); + boolean useLegacyParquetDecryption = delete.format() == FileFormat.PARQUET && delete.encryptionKeyMetadata().isPresent(); + Optional effectiveParquetFileDecryptionData = useLegacyParquetDecryption ? Optional.empty() : parquetFileDecryptionData; + TrinoInputFile inputFile = trinoInputFile; + if (delete.format() != FileFormat.PARQUET || useLegacyParquetDecryption) { + inputFile = decryptInputFileIfNeeded( + trinoInputFile, + delete.encryptionKeyMetadata(), + encryptionManager); + } + long fileSize = delete.fileSizeInBytes(); + if ((delete.format() != FileFormat.PARQUET || useLegacyParquetDecryption) && delete.encryptionKeyMetadata().isPresent()) { + try { + fileSize = inputFile.length(); + } + catch (IOException e) { + throw new UncheckedIOException(e); + } + } return createDataPageSource( session, - fileSystem.newInputFile(Location.of(delete.path()), delete.fileSizeInBytes()), + inputFile, 0, - delete.fileSizeInBytes(), - delete.fileSizeInBytes(), + fileSize, + fileSize, 0, "", IcebergFileFormat.fromIceberg(delete.format()), @@ -542,6 +600,7 @@ public ConnectorPageSource openDeleteFile( columns, tupleDomain, Optional.empty(), + effectiveParquetFileDecryptionData, "", ImmutableMap.of(), 0, @@ -549,6 +608,66 @@ public ConnectorPageSource openDeleteFile( .pageSource(); } + public Optional encryptionManager(Map tableProperties) + { + EncryptionManager encryptionManager = encryptionManagerFactory.createEncryptionManager(tableProperties); + if (encryptionManager instanceof PlaintextEncryptionManager) { + return Optional.empty(); + } + return Optional.of(encryptionManager); + } + + private static TrinoInputFile newInputFile(TrinoFileSystem fileSystem, String path, long fileSize, Optional keyMetadata, boolean useMetadataLength) + { + if (keyMetadata.isPresent() || !useMetadataLength) { + return fileSystem.newInputFile(Location.of(path)); + } + return fileSystem.newInputFile(Location.of(path), fileSize); + } + + private static TrinoInputFile decryptInputFileIfNeeded( + TrinoInputFile inputFile, + Optional keyMetadata, + Optional encryptionManager) + { + if (keyMetadata.isEmpty() || encryptionManager.isEmpty()) { + return inputFile; + } + byte[] metadataWithLength = keyMetadata.get(); + InputFile encryptedInputFile = new ForwardingInputFile(inputFile); + InputFile decryptedInputFile = encryptionManager.get().decrypt(EncryptedFiles.encryptedInput(encryptedInputFile, metadataWithLength)); + return new EncryptedTrinoInputFile(inputFile, decryptedInputFile); + } + + private static Optional parquetFileDecryptionProperties(Optional parquetFileDecryptionData) + { + return parquetFileDecryptionData.map(data -> FileDecryptionProperties.builder() + .withKeyRetriever(new StaticKeyRetriever(data.fileEncryptionKey())) + .withAadPrefix(data.fileAadPrefix()) + .build()); + } + + private record StaticKeyRetriever(byte[] fileEncryptionKey) + implements DecryptionKeyRetriever + { + private StaticKeyRetriever + { + requireNonNull(fileEncryptionKey, "fileEncryptionKey is null"); + } + + @Override + public Optional getColumnKey(ColumnPath columnPath, Optional keyMetadata) + { + return Optional.of(fileEncryptionKey); + } + + @Override + public Optional getFooterKey(Optional keyMetadata) + { + return Optional.of(fileEncryptionKey); + } + } + private ReaderPageSourceWithRowPositions createDataPageSource( ConnectorSession session, TrinoInputFile inputFile, @@ -562,6 +681,7 @@ private ReaderPageSourceWithRowPositions createDataPageSource( List dataColumns, TupleDomain predicate, Optional nameMapping, + Optional parquetFileDecryptionData, String partition, Map> partitionKeys, long dataSequenceNumber, @@ -615,6 +735,7 @@ private ReaderPageSourceWithRowPositions createDataPageSource( predicate, fileFormatDataSourceStats, nameMapping, + parquetFileDecryptionData, partition, partitionKeys, dataSequenceNumber, @@ -1005,6 +1126,7 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource( TupleDomain effectivePredicate, FileFormatDataSourceStats fileFormatDataSourceStats, Optional nameMapping, + Optional parquetFileDecryptionData, String partition, Map> partitionKeys, long dataSequenceNumber, @@ -1015,7 +1137,10 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource( ParquetDataSource dataSource = null; try { dataSource = createDataSource(inputFile, OptionalLong.of(fileSize), options, memoryContext, fileFormatDataSourceStats); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, options.getMaxFooterReadSize(), Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter( + dataSource, + options.getMaxFooterReadSize(), + parquetFileDecryptionProperties(parquetFileDecryptionData)); FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); if (nameMapping.isPresent() && !ParquetSchemaUtil.hasIds(fileSchema)) { @@ -1174,7 +1299,7 @@ else if (column.isBaseColumn()) { exception -> handleException(dataSourceId, exception), Optional.empty(), Optional.empty(), - Optional.empty()); + parquetMetadata.getDecryptionContext()); ConnectorPageSource pageSource = new ParquetPageSource(parquetReader); pageSource = transforms.build(pageSource); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProviderFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProviderFactory.java index 2c1fdb0583ab..f0ba9c9d2bbb 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProviderFactory.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergPageSourceProviderFactory.java @@ -19,6 +19,7 @@ import io.trino.plugin.base.metrics.FileFormatDataSourceStats; import io.trino.plugin.hive.orc.OrcReaderConfig; import io.trino.plugin.hive.parquet.ParquetReaderConfig; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorPageSourceProviderFactory; import io.trino.spi.type.TypeManager; @@ -34,6 +35,7 @@ public class IcebergPageSourceProviderFactory private final OrcReaderOptions orcReaderOptions; private final ParquetReaderOptions parquetReaderOptions; private final TypeManager typeManager; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; @Inject public IcebergPageSourceProviderFactory( @@ -42,7 +44,8 @@ public IcebergPageSourceProviderFactory( FileFormatDataSourceStats fileFormatDataSourceStats, OrcReaderConfig orcReaderConfig, ParquetReaderConfig parquetReaderConfig, - TypeManager typeManager) + TypeManager typeManager, + IcebergEncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); @@ -50,11 +53,12 @@ public IcebergPageSourceProviderFactory( this.orcReaderOptions = orcReaderConfig.toOrcReaderOptions(); this.parquetReaderOptions = parquetReaderConfig.toParquetReaderOptions(); this.typeManager = requireNonNull(typeManager, "typeManager is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override public IcebergPageSourceProvider createPageSourceProvider() { - return new IcebergPageSourceProvider(fileSystemFactory, fileIoFactory, fileFormatDataSourceStats, orcReaderOptions, parquetReaderOptions, typeManager); + return new IcebergPageSourceProvider(fileSystemFactory, fileIoFactory, fileFormatDataSourceStats, orcReaderOptions, parquetReaderOptions, typeManager, encryptionManagerFactory); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSortingFileWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSortingFileWriter.java index 223d62203945..c83793077508 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSortingFileWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSortingFileWriter.java @@ -25,6 +25,7 @@ import java.io.Closeable; import java.util.List; +import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -66,6 +67,12 @@ public FileMetrics getFileMetrics() return outputWriter.getFileMetrics(); } + @Override + public Optional getEncryptionKeyMetadata() + { + return outputWriter.getEncryptionKeyMetadata(); + } + @Override public long getWrittenBytes() { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplit.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplit.java index b6ea5d65b06f..99bf764ce115 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplit.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplit.java @@ -18,6 +18,7 @@ import com.fasterxml.jackson.annotation.JsonProperty; import com.google.common.base.MoreObjects.ToStringHelper; import com.google.common.collect.ImmutableList; +import io.airlift.slice.SizeOf; import io.trino.plugin.iceberg.delete.DeleteFile; import io.trino.spi.HostAddress; import io.trino.spi.SplitWeight; @@ -46,6 +47,8 @@ public class IcebergSplit private final long fileSize; private final long fileRecordCount; private final IcebergFileFormat fileFormat; + private final Optional encryptionKeyMetadata; + private final Optional parquetFileDecryptionData; private final Optional> partitionValues; private final int specId; private final String partitionDataJson; @@ -65,6 +68,8 @@ public IcebergSplit( @JsonProperty("fileRecordCount") long fileRecordCount, @JsonProperty("fileFormat") IcebergFileFormat fileFormat, @JsonProperty("specId") int specId, + @JsonProperty("parquetFileDecryptionData") Optional parquetFileDecryptionData, + @JsonProperty("encryptionKeyMetadata") Optional encryptionKeyMetadata, @JsonProperty("partitionDataJson") String partitionDataJson, @JsonProperty("deletes") List deletes, @JsonProperty("splitWeight") SplitWeight splitWeight, @@ -79,6 +84,8 @@ public IcebergSplit( fileSize, fileRecordCount, fileFormat, + Optional.ofNullable(encryptionKeyMetadata).orElse(Optional.empty()), + Optional.ofNullable(parquetFileDecryptionData).orElse(Optional.empty()), Optional.empty(), specId, partitionDataJson, @@ -97,6 +104,8 @@ public IcebergSplit( long fileSize, long fileRecordCount, IcebergFileFormat fileFormat, + Optional encryptionKeyMetadata, + Optional parquetFileDecryptionData, Optional> partitionValues, int specId, String partitionDataJson, @@ -113,6 +122,8 @@ public IcebergSplit( this.fileSize = fileSize; this.fileRecordCount = fileRecordCount; this.fileFormat = requireNonNull(fileFormat, "fileFormat is null"); + this.encryptionKeyMetadata = requireNonNull(encryptionKeyMetadata, "encryptionKeyMetadata is null"); + this.parquetFileDecryptionData = requireNonNull(parquetFileDecryptionData, "parquetFileDecryptionData is null"); this.partitionValues = requireNonNull(partitionValues, "partitionValues is null"); this.specId = specId; this.partitionDataJson = requireNonNull(partitionDataJson, "partitionDataJson is null"); @@ -167,6 +178,18 @@ public IcebergFileFormat getFileFormat() return fileFormat; } + @JsonProperty + public Optional getEncryptionKeyMetadata() + { + return encryptionKeyMetadata; + } + + @JsonProperty + public Optional getParquetFileDecryptionData() + { + return parquetFileDecryptionData; + } + @JsonProperty public int getSpecId() { @@ -228,6 +251,8 @@ public long getRetainedSizeInBytes() + SIZE_OF_LONG * 4 // start, length, fileSize, fileRecordCount + SIZE_OF_INT // specId + estimatedSizeOf(partitionDataJson) + + encryptionKeyMetadata.map(SizeOf::sizeOf).orElse(0L) + + parquetFileDecryptionData.map(ParquetFileDecryptionData::getRetainedSizeInBytes).orElse(0L) + estimatedSizeOf(deletes, DeleteFile::retainedSizeInBytes) + splitWeight.getRetainedSizeInBytes() + fileStatisticsDomain.getRetainedSizeInBytes(IcebergColumnHandle::getRetainedSizeInBytes) @@ -251,4 +276,22 @@ public String toString() } return helper.toString(); } + + public record ParquetFileDecryptionData(byte[] fileEncryptionKey, byte[] fileAadPrefix) + { + private static final int INSTANCE_SIZE = instanceSize(ParquetFileDecryptionData.class); + + public ParquetFileDecryptionData + { + requireNonNull(fileEncryptionKey, "fileEncryptionKey is null"); + requireNonNull(fileAadPrefix, "fileAadPrefix is null"); + } + + public long getRetainedSizeInBytes() + { + return INSTANCE_SIZE + + SizeOf.sizeOf(fileEncryptionKey) + + SizeOf.sizeOf(fileAadPrefix); + } + } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitManager.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitManager.java index 1a8c67c4ca8e..30ffce5bbcd8 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitManager.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitManager.java @@ -19,6 +19,7 @@ import io.airlift.units.Duration; import io.trino.filesystem.cache.CachingHostAddressProvider; import io.trino.plugin.base.classloader.ClassLoaderSafeConnectorSplitSource; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.functions.tablechanges.TableChangesFunctionHandle; import io.trino.plugin.iceberg.functions.tablechanges.TableChangesSplitSource; import io.trino.spi.connector.ConnectorSession; @@ -37,10 +38,14 @@ import org.apache.iceberg.Scan; import org.apache.iceberg.Snapshot; import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.metrics.MetricsReporter; import org.apache.iceberg.util.SnapshotUtil; +import java.util.Map; +import java.util.Optional; import java.util.concurrent.ExecutorService; import static io.trino.plugin.iceberg.IcebergSessionProperties.getDynamicFilteringWaitTimeout; @@ -56,6 +61,7 @@ public class IcebergSplitManager private final IcebergTransactionManager transactionManager; private final TypeManager typeManager; private final IcebergFileSystemFactory fileSystemFactory; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; private final ListeningExecutorService splitSourceExecutor; private final ExecutorService icebergPlanningExecutor; private final CachingHostAddressProvider cachingHostAddressProvider; @@ -65,6 +71,7 @@ public IcebergSplitManager( IcebergTransactionManager transactionManager, TypeManager typeManager, IcebergFileSystemFactory fileSystemFactory, + IcebergEncryptionManagerFactory encryptionManagerFactory, @ForIcebergSplitSource ListeningExecutorService splitSourceExecutor, @ForIcebergSplitManager ExecutorService icebergPlanningExecutor, CachingHostAddressProvider cachingHostAddressProvider) @@ -72,6 +79,7 @@ public IcebergSplitManager( this.transactionManager = requireNonNull(transactionManager, "transactionManager is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); this.splitSourceExecutor = requireNonNull(splitSourceExecutor, "splitSourceExecutor is null"); this.icebergPlanningExecutor = requireNonNull(icebergPlanningExecutor, "icebergPlanningExecutor is null"); this.cachingHostAddressProvider = requireNonNull(cachingHostAddressProvider, "cachingHostAddressProvider is null"); @@ -100,12 +108,14 @@ public ConnectorSplitSource getSplits( InMemoryMetricsReporter metricsReporter = new InMemoryMetricsReporter(); Scan scan = getScan(icebergMetadata, icebergTable, table, metricsReporter, icebergPlanningExecutor); + Optional encryptionManager = encryptionManager(table.getStorageProperties(), icebergTable); IcebergSplitSource splitSource = new IcebergSplitSource( fileSystemFactory, session, table, icebergTable, + encryptionManager, scan, table.getMaxScannedFileSize(), dynamicFilter, @@ -162,6 +172,7 @@ public ConnectorSplitSource getSplits( TableChangesSplitSource tableChangesSplitSource = new TableChangesSplitSource( icebergTable, + encryptionManager(functionHandle.storageProperties(), icebergTable), icebergTable.newIncrementalChangelogScan() .fromSnapshotExclusive(functionHandle.startSnapshotId()) .toSnapshot(functionHandle.endSnapshotId())); @@ -170,4 +181,18 @@ public ConnectorSplitSource getSplits( throw new IllegalStateException("Unknown table function: " + function); } + + private Optional encryptionManager(Map tableProperties, Table icebergTable) + { + EncryptionManager configuredEncryptionManager = encryptionManagerFactory.createEncryptionManager(tableProperties); + if (!(configuredEncryptionManager instanceof PlaintextEncryptionManager)) { + return Optional.of(configuredEncryptionManager); + } + + EncryptionManager tableEncryptionManager = icebergTable.encryption(); + if (tableEncryptionManager instanceof PlaintextEncryptionManager) { + return Optional.empty(); + } + return Optional.of(tableEncryptionManager); + } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index a9e1e84fe2d8..060d927ac5b7 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java @@ -32,6 +32,7 @@ import io.trino.filesystem.cache.CachingHostAddressProvider; import io.trino.plugin.base.metrics.DurationTiming; import io.trino.plugin.base.metrics.LongCount; +import io.trino.plugin.iceberg.IcebergSplit.ParquetFileDecryptionData; import io.trino.plugin.iceberg.delete.DeleteFile; import io.trino.plugin.iceberg.util.DataFileWithDeleteFiles; import io.trino.spi.SplitWeight; @@ -61,13 +62,21 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedInputFile; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.NativeEncryptionInputFile; +import org.apache.iceberg.encryption.NativeEncryptionKeyMetadata; import org.apache.iceberg.expressions.Expression; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.metrics.ScanMetricsResult; import org.apache.iceberg.metrics.ScanReport; import org.apache.iceberg.types.Type; +import org.apache.iceberg.util.ByteBuffers; import java.io.IOException; import java.io.UncheckedIOException; @@ -136,6 +145,8 @@ public class IcebergSplitSource private final ConnectorSession session; private final IcebergTableHandle tableHandle; private final Map fileIoProperties; + private final FileIO fileIo; + private final Optional encryptionManager; private final Scan tableScan; private final OptionalLong maxScannedFileSizeInBytes; private final Map fieldIdToType; @@ -190,6 +201,7 @@ public IcebergSplitSource( ConnectorSession session, IcebergTableHandle tableHandle, Table icebergTable, + Optional encryptionManager, Scan tableScan, Optional maxScannedFileSize, DynamicFilter dynamicFilter, @@ -205,7 +217,9 @@ public IcebergSplitSource( this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.session = requireNonNull(session, "session is null"); this.tableHandle = requireNonNull(tableHandle, "tableHandle is null"); - this.fileIoProperties = requireNonNull(icebergTable.io().properties(), "fileIoProperties is null"); + this.fileIoProperties = requireNonNull(IcebergUtil.getFileIoProperties(icebergTable), "fileIoProperties is null"); + this.fileIo = requireNonNull(icebergTable.io(), "fileIo is null"); + this.encryptionManager = requireNonNull(encryptionManager, "encryptionManager is null"); this.tableScan = requireNonNull(tableScan, "tableScan is null"); this.maxScannedFileSizeInBytes = maxScannedFileSize.isPresent() ? OptionalLong.of(maxScannedFileSize.orElseThrow().toBytes()) : OptionalLong.empty(); this.fieldIdToType = primitiveFieldTypes(tableScan.schema()); @@ -743,12 +757,14 @@ private IcebergSplit toIcebergSplit(FileScanTaskWithDomain taskWithDomain) task.file().fileSizeInBytes(), task.file().recordCount(), IcebergFileFormat.fromIceberg(task.file().format()), + Optional.ofNullable(task.file().keyMetadata()).map(ByteBuffers::toByteArray), + parquetFileDecryptionData(task), partitionValues, task.spec().specId(), PartitionData.toJson(task.file().partition()), task.deletes().stream() .peek(file -> verifyDeletionVectorReferencesDataFile(task, file)) - .map(DeleteFile::fromIceberg) + .map(this::toDeleteFile) .collect(toImmutableList()), SplitWeight.fromProportion(clamp(getSplitWeight(task), minimumAssignedSplitWeight, 1.0)), taskWithDomain.fileStatisticsDomain(), @@ -757,6 +773,66 @@ private IcebergSplit toIcebergSplit(FileScanTaskWithDomain taskWithDomain) task.file().firstRowId() == null ? OptionalLong.empty() : OptionalLong.of(task.file().firstRowId())); } + private DeleteFile toDeleteFile(org.apache.iceberg.DeleteFile deleteFile) + { + if (deleteFile.format() != FileFormat.PARQUET || deleteFile.keyMetadata() == null || encryptionManager.isEmpty()) { + return DeleteFile.fromIceberg(deleteFile, Optional.empty()); + } + + EncryptedInputFile encryptedInputFile = EncryptedFiles.encryptedInput( + fileIo.newInputFile(deleteFile.location()), + deleteFile.keyMetadata()); + return DeleteFile.fromIceberg(deleteFile, parquetFileDecryptionData(encryptedInputFile, encryptionManager.orElseThrow())); + } + + private Optional parquetFileDecryptionData(FileScanTask task) + { + if (task.file().format() != FileFormat.PARQUET || task.file().keyMetadata() == null || encryptionManager.isEmpty()) { + return Optional.empty(); + } + + EncryptedInputFile encryptedInputFile = EncryptedFiles.encryptedInput( + fileIo.newInputFile(task.file().location()), + task.file().keyMetadata()); + return parquetFileDecryptionData(encryptedInputFile, encryptionManager.orElseThrow()); + } + + @VisibleForTesting + public static Optional parquetFileDecryptionData( + EncryptedInputFile encryptedInputFile, + EncryptionManager encryptionManager) + { + InputFile inputFile; + try { + inputFile = encryptionManager.decrypt(encryptedInputFile); + } + catch (RuntimeException e) { + return Optional.empty(); + } + + if (!(inputFile instanceof NativeEncryptionInputFile nativeEncryptionInputFile)) { + return Optional.empty(); + } + + NativeEncryptionKeyMetadata nativeKeyMetadata; + try { + nativeKeyMetadata = nativeEncryptionInputFile.keyMetadata(); + } + catch (RuntimeException e) { + return Optional.empty(); + } + + ByteBuffer encryptionKey = nativeKeyMetadata.encryptionKey(); + ByteBuffer aadPrefix = nativeKeyMetadata.aadPrefix(); + if (encryptionKey == null || aadPrefix == null) { + return Optional.empty(); + } + + return Optional.of(new ParquetFileDecryptionData( + ByteBuffers.toByteArray(encryptionKey), + ByteBuffers.toByteArray(aadPrefix))); + } + private static void verifyDeletionVectorReferencesDataFile(FileScanTask task, org.apache.iceberg.DeleteFile deleteFile) { if (deleteFile.format() != FileFormat.PUFFIN || deleteFile.contentOffset() == null || deleteFile.contentSizeInBytes() == null) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableCredentials.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableCredentials.java index 19b3ea3bee18..94457e91cdc0 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableCredentials.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableCredentials.java @@ -19,6 +19,8 @@ import java.util.Map; +import static io.trino.plugin.iceberg.IcebergUtil.getFileIoProperties; + public record IcebergTableCredentials(Map fileIoProperties) implements ConnectorTableCredentials { @@ -29,6 +31,6 @@ public record IcebergTableCredentials(Map fileIoProperties) public static IcebergTableCredentials forFileIO(FileIO io) { - return new IcebergTableCredentials(io.properties()); + return new IcebergTableCredentials(getFileIoProperties(io)); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableProperties.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableProperties.java index 2c9994030bb3..e6829d11dbf0 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableProperties.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergTableProperties.java @@ -26,6 +26,7 @@ import io.trino.spi.type.ArrayType; import io.trino.spi.type.MapType; import io.trino.spi.type.TypeManager; +import org.apache.iceberg.TableProperties; import java.util.List; import java.util.Map; @@ -72,6 +73,8 @@ public class IcebergTableProperties public static final String OBJECT_STORE_LAYOUT_ENABLED_PROPERTY = "object_store_layout_enabled"; public static final String DATA_LOCATION_PROPERTY = "data_location"; public static final String EXTRA_PROPERTIES_PROPERTY = "extra_properties"; + public static final String ENCRYPTION_KEY_ID_PROPERTY = "encryption_key_id"; + public static final String ENCRYPTION_DATA_KEY_LENGTH_PROPERTY = "encryption_data_key_length"; public static final Set SUPPORTED_PROPERTIES = ImmutableSet.builder() .add(FILE_FORMAT_PROPERTY) @@ -89,6 +92,8 @@ public class IcebergTableProperties .add(DATA_LOCATION_PROPERTY) .add(EXTRA_PROPERTIES_PROPERTY) .add(PARQUET_BLOOM_FILTER_COLUMNS_PROPERTY) + .add(ENCRYPTION_KEY_ID_PROPERTY) + .add(ENCRYPTION_DATA_KEY_LENGTH_PROPERTY) .build(); // These properties are used by Trino or Iceberg internally and cannot be set directly by users through extra_properties @@ -98,6 +103,8 @@ public class IcebergTableProperties .add(ORC_BLOOM_FILTER_FPP) .add(DEFAULT_FILE_FORMAT) .add(FORMAT_VERSION) + .add(TableProperties.ENCRYPTION_TABLE_KEY) + .add(TableProperties.ENCRYPTION_DEK_LENGTH) .build(); private final List> tableProperties; @@ -235,6 +242,17 @@ public IcebergTableProperties( "File system location URI for the table's data files", null, false)) + .add(stringProperty( + ENCRYPTION_KEY_ID_PROPERTY, + "Iceberg table encryption key id", + null, + false)) + .add(integerProperty( + ENCRYPTION_DATA_KEY_LENGTH_PROPERTY, + "Iceberg table encryption data key length in bytes", + null, + IcebergTableProperties::validateEncryptionDataKeyLength, + false)) .build(); checkState(SUPPORTED_PROPERTIES.containsAll(tableProperties.stream() @@ -283,6 +301,16 @@ public static int getFormatVersion(Map tableProperties) return (int) tableProperties.get(FORMAT_VERSION_PROPERTY); } + public static Optional getEncryptionKeyId(Map tableProperties) + { + return Optional.ofNullable((String) tableProperties.get(ENCRYPTION_KEY_ID_PROPERTY)); + } + + public static Optional getEncryptionDataKeyLength(Map tableProperties) + { + return Optional.ofNullable((Integer) tableProperties.get(ENCRYPTION_DATA_KEY_LENGTH_PROPERTY)); + } + private static void validateFormatVersion(int version) { if (version < FORMAT_VERSION_SUPPORT_MIN || version > FORMAT_VERSION_SUPPORT_MAX) { @@ -342,6 +370,13 @@ private static void validateOrcBloomFilterFpp(double fpp) } } + private static void validateEncryptionDataKeyLength(int length) + { + if (length != 16 && length != 24 && length != 32) { + throw new TrinoException(INVALID_TABLE_PROPERTY, "encryption_data_key_length must be 16, 24, or 32 bytes"); + } + } + public static List getParquetBloomFilterColumns(Map tableProperties) { List parquetBloomFilterColumns = (List) tableProperties.get(PARQUET_BLOOM_FILTER_COLUMNS_PROPERTY); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java index 40b871ee1f2b..80a0930a0558 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergUtil.java @@ -61,6 +61,7 @@ import org.apache.iceberg.ContentFile; import org.apache.iceberg.FileFormat; import org.apache.iceberg.FileScanTask; +import org.apache.iceberg.HasTableOperations; import org.apache.iceberg.HistoryEntry; import org.apache.iceberg.ManifestFile; import org.apache.iceberg.ManifestFiles; @@ -76,6 +77,7 @@ import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableOperations; import org.apache.iceberg.Transaction; +import org.apache.iceberg.encryption.EncryptingFileIO; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.types.Type.PrimitiveType; @@ -86,6 +88,7 @@ import java.io.IOException; import java.io.UncheckedIOException; import java.lang.invoke.MethodHandle; +import java.lang.reflect.Field; import java.math.BigDecimal; import java.math.BigInteger; import java.nio.ByteBuffer; @@ -140,6 +143,8 @@ import static io.trino.plugin.iceberg.IcebergTableProperties.COMPRESSION_CODEC; import static io.trino.plugin.iceberg.IcebergTableProperties.DATA_LOCATION_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.DELETE_AFTER_COMMIT_ENABLED; +import static io.trino.plugin.iceberg.IcebergTableProperties.ENCRYPTION_DATA_KEY_LENGTH_PROPERTY; +import static io.trino.plugin.iceberg.IcebergTableProperties.ENCRYPTION_KEY_ID_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.FILE_FORMAT_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.FORMAT_VERSION_PROPERTY; import static io.trino.plugin.iceberg.IcebergTableProperties.LOCATION_PROPERTY; @@ -205,6 +210,8 @@ import static org.apache.iceberg.TableProperties.COMMIT_NUM_RETRIES; 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_TABLE_KEY; import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.METADATA_DELETE_AFTER_COMMIT_ENABLED; import static org.apache.iceberg.TableProperties.METADATA_PREVIOUS_VERSIONS_MAX; @@ -250,6 +257,7 @@ public final class IcebergUtil // - v0.gz.metadata.json // - v0.metadata.json.gz private static final Pattern HADOOP_GENERATED_METADATA_FILE_NAME_PATTERN = Pattern.compile("v(?\\d+)(?\\.[a-zA-Z0-9]+)?" + Pattern.quote(METADATA_FILE_EXTENSION) + "(?\\.[a-zA-Z0-9]+)?"); + private static final Field ENCRYPTING_FILE_IO_DELEGATE_FIELD = getEncryptingFileIoDelegateField(); private IcebergUtil() {} @@ -283,6 +291,47 @@ public static BaseTable getIcebergTableWithMetadata( return new BaseTable(operations, quotedTableName(table), TRINO_METRICS_REPORTER); } + public static Map getFileIoProperties(Table table) + { + requireNonNull(table, "table is null"); + if (table instanceof HasTableOperations hasTableOperations && hasTableOperations.operations() instanceof IcebergTableOperations icebergTableOperations) { + return icebergTableOperations.fileIoProperties(); + } + return getFileIoProperties(table.io()); + } + + public static Map getFileIoProperties(FileIO fileIo) + { + requireNonNull(fileIo, "fileIo is null"); + try { + return fileIo.properties(); + } + catch (UnsupportedOperationException e) { + // TODO: Remove this catch block once https://github.com/apache/iceberg/pull/15289 is released. + if (fileIo instanceof EncryptingFileIO encryptingFileIo) { + try { + return ((FileIO) ENCRYPTING_FILE_IO_DELEGATE_FIELD.get(encryptingFileIo)).properties(); + } + catch (IllegalAccessException reflectionException) { + throw new RuntimeException("Failed to extract wrapped FileIO from EncryptingFileIO", reflectionException); + } + } + throw e; + } + } + + private static Field getEncryptingFileIoDelegateField() + { + try { + Field field = EncryptingFileIO.class.getDeclaredField("io"); + field.setAccessible(true); + return field; + } + catch (ReflectiveOperationException e) { + throw new RuntimeException("Failed to initialize EncryptingFileIO delegate field", e); + } + } + public static List getPartitionColumns(Table table, TypeManager typeManager) { Set projectedIds = table.spec().fields().stream().map(PartitionField::sourceId).collect(toImmutableSet()); @@ -362,6 +411,11 @@ public static Map getIcebergTableProperties(BaseTable icebergTab int formatVersion = formatVersion(icebergTable); properties.put(FORMAT_VERSION_PROPERTY, formatVersion); + Optional.ofNullable(icebergTable.properties().get(ENCRYPTION_TABLE_KEY)) + .ifPresent(value -> properties.put(ENCRYPTION_KEY_ID_PROPERTY, value)); + Optional.ofNullable(icebergTable.properties().get(ENCRYPTION_DEK_LENGTH)) + .ifPresent(value -> properties.put(ENCRYPTION_DATA_KEY_LENGTH_PROPERTY, Integer.parseInt(value))); + if (icebergTable.properties().containsKey(COMMIT_NUM_RETRIES)) { int commitNumRetries = parseInt(icebergTable.properties().get(COMMIT_NUM_RETRIES)); properties.put(MAX_COMMIT_RETRY, commitNumRetries); @@ -996,6 +1050,14 @@ public static Map createTableProperties(ConnectorTableMetadata t propertiesBuilder.put(WRITE_DATA_LOCATION, location); }); + Optional encryptionKeyId = IcebergTableProperties.getEncryptionKeyId(tableMetadata.getProperties()); + Optional encryptionDataKeyLength = IcebergTableProperties.getEncryptionDataKeyLength(tableMetadata.getProperties()); + if (encryptionDataKeyLength.isPresent() && encryptionKeyId.isEmpty()) { + throw new TrinoException(INVALID_TABLE_PROPERTY, "encryption_data_key_length requires encryption_key_id"); + } + encryptionKeyId.ifPresent(value -> propertiesBuilder.put(ENCRYPTION_TABLE_KEY, value)); + encryptionDataKeyLength.ifPresent(value -> propertiesBuilder.put(ENCRYPTION_DEK_LENGTH, Integer.toString(value))); + // iceberg ORC format bloom filter properties used by create table List orcBloomFilterColumns = IcebergTableProperties.getOrcBloomFilterColumns(tableMetadata.getProperties()); if (!orcBloomFilterColumns.isEmpty()) { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java index 9c22434311c0..2c172168ae10 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/AbstractIcebergTableOperations.java @@ -22,6 +22,7 @@ import io.trino.metastore.HiveType; import io.trino.metastore.StorageFormat; import io.trino.plugin.iceberg.IcebergExceptions; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.util.HiveSchemaUtil; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; @@ -29,6 +30,9 @@ import jakarta.annotation.Nullable; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; +import org.apache.iceberg.encryption.EncryptingFileIO; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; @@ -65,6 +69,7 @@ import static org.apache.iceberg.BaseMetastoreTableOperations.METADATA_LOCATION_PROP; import static org.apache.iceberg.CatalogUtil.deleteRemovedMetadataFiles; import static org.apache.iceberg.TableMetadataParser.getFileExtension; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; import static org.apache.iceberg.TableProperties.METADATA_COMPRESSION; import static org.apache.iceberg.TableProperties.METADATA_COMPRESSION_DEFAULT; import static org.apache.iceberg.TableProperties.WRITE_METADATA_LOCATION; @@ -84,15 +89,20 @@ public abstract class AbstractIcebergTableOperations protected final String tableName; protected final Optional owner; protected final Optional location; - private final FileIO fileIo; + protected final FileIO fileIo; + protected final IcebergEncryptionManagerFactory encryptionManagerFactory; protected TableMetadata currentMetadata; protected String currentMetadataLocation; protected boolean shouldRefresh = true; protected OptionalInt version = OptionalInt.empty(); + private String tableKeyId; + private EncryptionManager encryptionManager; + private FileIO encryptingFileIo; protected AbstractIcebergTableOperations( FileIO fileIo, + IcebergEncryptionManagerFactory encryptionManagerFactory, ConnectorSession session, String database, String table, @@ -100,6 +110,7 @@ protected AbstractIcebergTableOperations( Optional location) { this.fileIo = requireNonNull(fileIo, "fileIo is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); this.session = requireNonNull(session, "session is null"); this.database = requireNonNull(database, "database is null"); this.tableName = requireNonNull(table, "table is null"); @@ -115,6 +126,7 @@ public void initializeFromMetadata(TableMetadata tableMetadata) currentMetadataLocation = tableMetadata.metadataFileLocation(); shouldRefresh = false; version = OptionalInt.of(parseVersion(Location.of(currentMetadataLocation).fileName())); + updateEncryptionState(tableMetadata); } @Override @@ -192,7 +204,31 @@ public void commit(@Nullable TableMetadata base, TableMetadata metadata) @Override public FileIO io() { - return fileIo; + if (!isEncryptedTable()) { + return fileIo; + } + if (encryptingFileIo == null) { + encryptingFileIo = EncryptingFileIO.combine(fileIo, encryption()); + } + return encryptingFileIo; + } + + @Override + public Map fileIoProperties() + { + return fileIo.properties(); + } + + @Override + public EncryptionManager encryption() + { + if (!isEncryptedTable()) { + return PlaintextEncryptionManager.instance(); + } + if (encryptionManager == null) { + encryptionManager = encryptionManagerFactory.createEncryptionManager(currentMetadata); + } + return encryptionManager; } @Override @@ -284,6 +320,19 @@ protected void refreshFromMetadataLocation(String newLocation, Function fileIoProperties() + { + return io().properties(); + } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperations.java index 5c784badb4dd..cc32e7c713f6 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperations.java @@ -20,6 +20,7 @@ import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.MetastoreUtil; import io.trino.plugin.iceberg.catalog.hms.AbstractMetastoreTableOperations; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.TableMetadata; @@ -43,6 +44,7 @@ public class FileMetastoreTableOperations { public FileMetastoreTableOperations( FileIO fileIo, + IcebergEncryptionManagerFactory encryptionManagerFactory, CachingHiveMetastore metastore, ConnectorSession session, String database, @@ -50,7 +52,7 @@ public FileMetastoreTableOperations( Optional owner, Optional location) { - super(fileIo, metastore, session, database, table, owner, location); + super(fileIo, encryptionManagerFactory, metastore, session, database, table, owner, location); } @Override diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperationsProvider.java index 6f99baaffc57..99d16564fa41 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/file/FileMetastoreTableOperationsProvider.java @@ -19,6 +19,7 @@ import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorSession; @@ -32,14 +33,17 @@ public class FileMetastoreTableOperationsProvider { private final TrinoFileSystemFactory fileSystemFactory; private final ForwardingFileIoFactory fileIoFactory; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; @Inject public FileMetastoreTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, - ForwardingFileIoFactory fileIoFactory) + ForwardingFileIoFactory fileIoFactory, + IcebergEncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override @@ -53,6 +57,7 @@ public IcebergTableOperations createTableOperations( { return new FileMetastoreTableOperations( fileIoFactory.create(fileSystemFactory.create(session), isUseFileSizeFromMetadata(session)), + encryptionManagerFactory, ((TrinoHiveCatalog) catalog).getMetastore(), session, database, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java index 2f09c9e3e2e4..65e63e32bf7b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperations.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableMap; import io.trino.plugin.iceberg.UnknownTableTypeException; import io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.SchemaTableName; @@ -74,13 +75,14 @@ protected GlueIcebergTableOperations( StatsRecordingGlueClient glueClient, GetGlueTable getGlueTable, FileIO fileIo, + IcebergEncryptionManagerFactory encryptionManagerFactory, ConnectorSession session, String database, String table, Optional owner, Optional location) { - super(fileIo, session, database, table, owner, location); + super(fileIo, encryptionManagerFactory, session, database, table, owner, location); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.cacheTableMetadata = cacheTableMetadata; this.glueClient = requireNonNull(glueClient, "glueClient is null"); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperationsProvider.java index 2434082b8bda..47b531115378 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/glue/GlueIcebergTableOperationsProvider.java @@ -19,6 +19,7 @@ import io.trino.plugin.iceberg.catalog.IcebergTableOperations; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.type.TypeManager; @@ -37,6 +38,7 @@ public class GlueIcebergTableOperationsProvider private final TypeManager typeManager; private final boolean cacheTableMetadata; private final StatsRecordingGlueClient glueClient; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; @Inject public GlueIcebergTableOperationsProvider( @@ -45,13 +47,15 @@ public GlueIcebergTableOperationsProvider( TypeManager typeManager, IcebergGlueCatalogConfig catalogConfig, GlueMetastoreStats stats, - GlueClient glueClient) + GlueClient glueClient, + IcebergEncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); this.typeManager = requireNonNull(typeManager, "typeManager is null"); this.cacheTableMetadata = catalogConfig.isCacheTableMetadata(); this.glueClient = new StatsRecordingGlueClient(glueClient, stats); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override @@ -71,6 +75,7 @@ public IcebergTableOperations createTableOperations( // the GetTableRequest is issued once per table. ((TrinoGlueCatalog) catalog)::getTable, fileIoFactory.create(fileSystemFactory.create(session), isUseFileSizeFromMetadata(session)), + encryptionManagerFactory, session, database, table, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/AbstractMetastoreTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/AbstractMetastoreTableOperations.java index fae8b0b0b7be..024e4eb8e9b8 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/AbstractMetastoreTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/AbstractMetastoreTableOperations.java @@ -21,6 +21,7 @@ import io.trino.plugin.iceberg.CreateTableException; import io.trino.plugin.iceberg.UnknownTableTypeException; import io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.TableNotFoundException; @@ -57,6 +58,7 @@ public abstract class AbstractMetastoreTableOperations protected AbstractMetastoreTableOperations( FileIO fileIo, + IcebergEncryptionManagerFactory encryptionManagerFactory, CachingHiveMetastore metastore, ConnectorSession session, String database, @@ -64,7 +66,7 @@ protected AbstractMetastoreTableOperations( Optional owner, Optional location) { - super(fileIo, session, database, table, owner, location); + super(fileIo, encryptionManagerFactory, session, database, table, owner, location); this.metastore = requireNonNull(metastore, "metastore is null"); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java index a736b7ff0756..05444cc9f043 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperations.java @@ -22,6 +22,7 @@ import io.trino.metastore.cache.CachingHiveMetastore; import io.trino.plugin.hive.metastore.MetastoreUtil; import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.TableNotFoundException; import org.apache.iceberg.TableMetadata; @@ -56,6 +57,7 @@ public class HiveMetastoreTableOperations public HiveMetastoreTableOperations( FileIO fileIo, + IcebergEncryptionManagerFactory encryptionManagerFactory, CachingHiveMetastore metastore, ThriftMetastore thriftMetastore, boolean lockingEnabled, @@ -65,7 +67,7 @@ public HiveMetastoreTableOperations( Optional owner, Optional location) { - super(fileIo, metastore, session, database, table, owner, location); + super(fileIo, encryptionManagerFactory, metastore, session, database, table, owner, location); this.thriftMetastore = requireNonNull(thriftMetastore, "thriftMetastore is null"); this.lockingEnabled = lockingEnabled; } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java index d66d43a10c37..d73dfa940b33 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/HiveMetastoreTableOperationsProvider.java @@ -19,6 +19,7 @@ import io.trino.plugin.iceberg.catalog.IcebergTableOperations; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorSession; @@ -34,18 +35,21 @@ public class HiveMetastoreTableOperationsProvider private final ForwardingFileIoFactory fileIoFactory; private final ThriftMetastoreFactory thriftMetastoreFactory; private final boolean lockingEnabled; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; @Inject public HiveMetastoreTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, ThriftMetastoreFactory thriftMetastoreFactory, - IcebergHiveCatalogConfig metastoreConfig) + IcebergHiveCatalogConfig metastoreConfig, + IcebergEncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); this.thriftMetastoreFactory = requireNonNull(thriftMetastoreFactory, "thriftMetastoreFactory is null"); this.lockingEnabled = metastoreConfig.getLockingEnabled(); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override @@ -59,6 +63,7 @@ public IcebergTableOperations createTableOperations( { return new HiveMetastoreTableOperations( fileIoFactory.create(fileSystemFactory.create(session), isUseFileSizeFromMetadata(session)), + encryptionManagerFactory, ((TrinoHiveCatalog) catalog).getMetastore(), thriftMetastoreFactory.createMetastore(Optional.of(session.getIdentity())), lockingEnabled, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/IcebergJdbcTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/IcebergJdbcTableOperations.java index 458435828543..18affd9c9b21 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/IcebergJdbcTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/IcebergJdbcTableOperations.java @@ -14,6 +14,7 @@ package io.trino.plugin.iceberg.catalog.jdbc; import io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.TableNotFoundException; import org.apache.iceberg.TableMetadata; @@ -32,6 +33,7 @@ public class IcebergJdbcTableOperations public IcebergJdbcTableOperations( FileIO fileIo, + IcebergEncryptionManagerFactory encryptionManagerFactory, IcebergJdbcClient jdbcClient, ConnectorSession session, String database, @@ -39,7 +41,7 @@ public IcebergJdbcTableOperations( Optional owner, Optional location) { - super(fileIo, session, database, table, owner, location); + super(fileIo, encryptionManagerFactory, session, database, table, owner, location); this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null"); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/IcebergJdbcTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/IcebergJdbcTableOperationsProvider.java index 3adb3b2c70be..a7159f437741 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/IcebergJdbcTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/jdbc/IcebergJdbcTableOperationsProvider.java @@ -18,6 +18,7 @@ import io.trino.plugin.iceberg.catalog.IcebergTableOperations; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorSession; @@ -32,16 +33,19 @@ public class IcebergJdbcTableOperationsProvider private final TrinoFileSystemFactory fileSystemFactory; private final ForwardingFileIoFactory fileIoFactory; private final IcebergJdbcClient jdbcClient; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; @Inject public IcebergJdbcTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, - IcebergJdbcClient jdbcClient) + IcebergJdbcClient jdbcClient, + IcebergEncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); this.jdbcClient = requireNonNull(jdbcClient, "jdbcClient is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override @@ -55,6 +59,7 @@ public IcebergTableOperations createTableOperations( { return new IcebergJdbcTableOperations( fileIoFactory.create(fileSystemFactory.create(session), isUseFileSizeFromMetadata(session)), + encryptionManagerFactory, jdbcClient, session, database, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperations.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperations.java index 02297f1e38b9..45632977f982 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperations.java @@ -14,6 +14,7 @@ package io.trino.plugin.iceberg.catalog.nessie; import io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.SchemaTableName; @@ -48,13 +49,14 @@ public class IcebergNessieTableOperations protected IcebergNessieTableOperations( NessieIcebergClient nessieClient, FileIO fileIo, + IcebergEncryptionManagerFactory encryptionManagerFactory, ConnectorSession session, String database, String table, Optional owner, Optional location) { - super(fileIo, session, database, table, owner, location); + super(fileIo, encryptionManagerFactory, session, database, table, owner, location); this.nessieClient = requireNonNull(nessieClient, "nessieClient is null"); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperationsProvider.java index 179a845ebca9..c3d703caa291 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/nessie/IcebergNessieTableOperationsProvider.java @@ -18,6 +18,7 @@ import io.trino.plugin.iceberg.catalog.IcebergTableOperations; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.nessie.NessieIcebergClient; @@ -33,13 +34,19 @@ public class IcebergNessieTableOperationsProvider private final TrinoFileSystemFactory fileSystemFactory; private final ForwardingFileIoFactory fileIoFactory; private final NessieIcebergClient nessieClient; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; @Inject - public IcebergNessieTableOperationsProvider(TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, NessieIcebergClient nessieClient) + public IcebergNessieTableOperationsProvider( + TrinoFileSystemFactory fileSystemFactory, + ForwardingFileIoFactory fileIoFactory, + NessieIcebergClient nessieClient, + IcebergEncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); this.nessieClient = requireNonNull(nessieClient, "nessieClient is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override @@ -54,6 +61,7 @@ public IcebergTableOperations createTableOperations( return new IcebergNessieTableOperations( nessieClient, fileIoFactory.create(fileSystemFactory.create(session), isUseFileSizeFromMetadata(session)), + encryptionManagerFactory, session, database, table, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java index 600b4a59aa39..aac013f6ba9e 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/rest/TrinoRestCatalog.java @@ -507,7 +507,7 @@ private void purgeBigLakeTable(ConnectorSession session, SchemaTableName schemaT // So log the exception and continue with deleting the table location log.warn(e, "Failed to delete table data referenced by metadata"); } - deleteTableDirectory(fileSystemFactory.create(session.getIdentity(), table.io().properties()), schemaTableName, table.location()); + deleteTableDirectory(fileSystemFactory.create(session.getIdentity(), fileIoProperties(table)), schemaTableName, table.location()); } private static void deleteTableDirectory(TrinoFileSystem fileSystem, SchemaTableName schemaTableName, String tableLocation) @@ -520,6 +520,11 @@ private static void deleteTableDirectory(TrinoFileSystem fileSystem, SchemaTable } } + private static Map fileIoProperties(Table table) + { + return IcebergUtil.getFileIoProperties(table); + } + private void purgeTable(ConnectorSession session, SchemaTableName schemaTableName) { try { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/snowflake/SnowflakeIcebergTableOperationsProvider.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/snowflake/SnowflakeIcebergTableOperationsProvider.java index 3a30e478d031..4560f53e8674 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/snowflake/SnowflakeIcebergTableOperationsProvider.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/snowflake/SnowflakeIcebergTableOperationsProvider.java @@ -18,6 +18,7 @@ import io.trino.plugin.iceberg.catalog.IcebergTableOperations; import io.trino.plugin.iceberg.catalog.IcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.snowflake.SnowflakeIcebergTableOperations; @@ -32,17 +33,20 @@ public class SnowflakeIcebergTableOperationsProvider { private final TrinoFileSystemFactory fileSystemFactory; private final ForwardingFileIoFactory fileIoFactory; + private final IcebergEncryptionManagerFactory encryptionManagerFactory; private final String snowflakeDatabase; @Inject public SnowflakeIcebergTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, + IcebergEncryptionManagerFactory encryptionManagerFactory, IcebergSnowflakeCatalogConfig icebergSnowflakeCatalogConfig) { this.snowflakeDatabase = requireNonNull(icebergSnowflakeCatalogConfig.getDatabase(), "database is null"); this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override @@ -57,6 +61,7 @@ public IcebergTableOperations createTableOperations( return new SnowflakeIcebergTableOperations( (TrinoSnowflakeCatalog) catalog, fileIoFactory.create(fileSystemFactory.create(session), isUseFileSizeFromMetadata(session)), + encryptionManagerFactory, session, snowflakeDatabase, database, diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DefaultDeletionVectorWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DefaultDeletionVectorWriter.java index 09a530d22597..697cba35ee1a 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DefaultDeletionVectorWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DefaultDeletionVectorWriter.java @@ -21,11 +21,14 @@ import io.trino.filesystem.Location; import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoInput; +import io.trino.filesystem.TrinoInputFile; import io.trino.plugin.base.util.Closables; import io.trino.plugin.iceberg.IcebergColumnHandle; import io.trino.plugin.iceberg.IcebergFileSystemFactory; import io.trino.plugin.iceberg.IcebergPageSourceProviderFactory; import io.trino.plugin.iceberg.IcebergTableHandle; +import io.trino.plugin.iceberg.fileio.EncryptedTrinoInputFile; +import io.trino.plugin.iceberg.fileio.ForwardingInputFile; import io.trino.spi.NodeVersion; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorPageSource; @@ -41,7 +44,12 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.RowDelta; import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.io.OutputFile; import org.apache.iceberg.puffin.Blob; @@ -51,11 +59,13 @@ import org.apache.iceberg.util.ContentFileUtil; import java.io.IOException; +import java.nio.ByteBuffer; import java.util.ArrayList; import java.util.Collection; import java.util.HashMap; import java.util.List; import java.util.Map; +import java.util.Optional; import java.util.Set; import java.util.UUID; import java.util.function.Function; @@ -64,6 +74,7 @@ import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_BAD_DATA; import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_WRITER_DATA_ERROR; import static io.trino.plugin.iceberg.IcebergUtil.getColumnHandle; +import static io.trino.plugin.iceberg.IcebergUtil.getFileIoProperties; import static io.trino.plugin.iceberg.IcebergUtil.getLocationProvider; import static java.lang.Math.toIntExact; import static java.util.Objects.requireNonNull; @@ -72,6 +83,7 @@ import static org.apache.iceberg.MetadataColumns.DELETE_FILE_PATH; import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS; import static org.apache.iceberg.puffin.StandardBlobTypes.DV_V1; +import static org.apache.iceberg.util.ByteBuffers.toByteArray; public class DefaultDeletionVectorWriter implements DeletionVectorWriter @@ -109,6 +121,7 @@ public void writeDeletionVectors( RowDelta rowDelta) { long snapshotId = table.getSnapshotId().orElseThrow(() -> new TrinoException(ICEBERG_BAD_DATA, "Missing base snapshot id for v3 deletion vector rewrite")); + Optional encryptionManager = encryptionManager(icebergTable); // deletion vector info may contain multiple entries for the same data file; merge them here Map deletionVectorBuilders = deletionVectorInfos.stream().collect(toMap( @@ -119,9 +132,13 @@ public void writeDeletionVectors( ExistingDeletes existingDeletes = getExistingDeletesByMetadataOnly(icebergTable, snapshotId, deletionVectorBuilders.keySet()); // merge existing deletion vectors into the new ones - TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), icebergTable.io().properties()); + TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), getFileIoProperties(icebergTable)); existingDeletes.deletionVectors().forEach((dataFilePath, deleteFile) -> { - try (TrinoInput input = fileSystem.newInputFile(Location.of(deleteFile.location()), deleteFile.fileSizeInBytes()).newInput()) { + TrinoInputFile inputFile = decryptInputFileIfNeeded( + newInputFile(fileSystem, deleteFile, encryptionManager), + deleteFile, + encryptionManager); + try (TrinoInput input = inputFile.newInput()) { Slice data = input.readFully(deleteFile.contentOffset(), toIntExact(deleteFile.contentSizeInBytes())); deletionVectorBuilders.get(dataFilePath).deserialize(data); } @@ -142,7 +159,7 @@ public void writeDeletionVectors( deletionVectorsWithLegacyDelete.forEach((dataFilePath, deletionVector) -> { Collection deleteFiles = existingDeletes.fileScopedDeletes().get(dataFilePath); for (DeleteFile deleteFile : deleteFiles) { - try (ConnectorPageSource connectorPageSource = openDeleteFilePageSource(session, deleteFile, fileSystem)) { + try (ConnectorPageSource connectorPageSource = openDeleteFilePageSource(session, deleteFile, fileSystem, encryptionManager)) { PositionDeleteReader.readSingleFilePositionDeletes(connectorPageSource, deletionVector::add); } catch (IOException e) { @@ -153,7 +170,7 @@ public void writeDeletionVectors( // process the partition-scoped delete files for (DeleteFile deleteFile : existingDeletes.partitionScopedDeletes()) { - try (ConnectorPageSource connectorPageSource = openDeleteFilePageSource(session, deleteFile, fileSystem)) { + try (ConnectorPageSource connectorPageSource = openDeleteFilePageSource(session, deleteFile, fileSystem, encryptionManager)) { PositionDeleteReader.readMultiFilePositionDeletes(connectorPageSource, (dataFilePath, position) -> { DeletionVector.Builder deletionVector = deletionVectorsWithLegacyDelete.get(dataFilePath); if (deletionVector != null) { @@ -180,7 +197,7 @@ public void writeDeletionVectors( LocationProvider locationProvider = getLocationProvider(table.getSchemaTableName(), table.getTableLocation(), table.getStorageProperties()); // write deletion vectors to a puffin file and delete files to the row delta - writeDeletionVectorsPuffin(session, icebergTable, locationProvider, deletionVectorInfos, deletionVectors, trinoVersion) + writeDeletionVectorsPuffin(session, icebergTable, locationProvider, deletionVectorInfos, deletionVectors, trinoVersion, encryptionManager) .forEach(rowDelta::addDeletes); // remove existing DVs and file-scoped position deletes @@ -242,7 +259,8 @@ private static List writeDeletionVectorsPuffin( LocationProvider locationProvider, List deletionVectorInfos, Map deletionVectors, - String trinoVersion) + String trinoVersion, + Optional encryptionManager) { if (deletionVectors.isEmpty()) { return List.of(); @@ -253,12 +271,20 @@ private static List writeDeletionVectorsPuffin( FileIO fileIO = icebergTable.io(); OutputFile outputFile = fileIO.newOutputFile(puffinPath); + Optional keyMetadata = Optional.empty(); + if (encryptionManager.isPresent()) { + EncryptedOutputFile encryptedOutputFile = encryptionManager.get().encrypt(outputFile); + outputFile = encryptedOutputFile.encryptingOutputFile(); + keyMetadata = Optional.ofNullable(encryptedOutputFile.keyMetadata().buffer()); + } try { try (PuffinWriter writer = Puffin.write(outputFile).createdBy("Trino version " + trinoVersion).build()) { deletionVectors.forEach((referencedDataFile, deletionVector) -> writer.add(createDeletionVectorBlob(referencedDataFile, deletionVector))); writer.finish(); + Optional encryptionKeyMetadata = keyMetadata + .map(buffer -> toByteArray(buffer)); Map partitionInfo = deletionVectorInfos.stream() .collect(toMap( @@ -290,6 +316,7 @@ private static List writeDeletionVectorsPuffin( .withContentOffset(meta.offset()) .withContentSizeInBytes(meta.length()) .withRecordCount(cardinality); + encryptionKeyMetadata.ifPresent(bytes -> deleteBuilder.withEncryptionKeyMetadata(ByteBuffer.wrap(bytes))); deletionVectorInfo.partitionData().ifPresent(deleteBuilder::withPartition); deleteFiles.add(deleteBuilder.build()); } @@ -308,14 +335,50 @@ private static List writeDeletionVectorsPuffin( } } - private ConnectorPageSource openDeleteFilePageSource(ConnectorSession session, DeleteFile deleteFile, TrinoFileSystem fileSystem) + private ConnectorPageSource openDeleteFilePageSource( + ConnectorSession session, + DeleteFile deleteFile, + TrinoFileSystem fileSystem, + Optional encryptionManager) { return pageSourceProviderFactory.createPageSourceProvider().openDeleteFile( session, fileSystem, - io.trino.plugin.iceberg.delete.DeleteFile.fromIceberg(deleteFile), + io.trino.plugin.iceberg.delete.DeleteFile.fromIceberg(deleteFile, Optional.empty()), List.of(deleteFilePathColumnHandle, deleteFilePositionColumnHandle), - TupleDomain.all()); + TupleDomain.all(), + encryptionManager); + } + + private static Optional encryptionManager(Table icebergTable) + { + EncryptionManager encryptionManager = icebergTable.encryption(); + if (encryptionManager instanceof PlaintextEncryptionManager) { + return Optional.empty(); + } + return Optional.of(encryptionManager); + } + + private static TrinoInputFile newInputFile(TrinoFileSystem fileSystem, DeleteFile deleteFile, Optional encryptionManager) + { + if (encryptionManager.isPresent() && deleteFile.keyMetadata() != null) { + return fileSystem.newInputFile(Location.of(deleteFile.location())); + } + return fileSystem.newInputFile(Location.of(deleteFile.location()), deleteFile.fileSizeInBytes()); + } + + private static TrinoInputFile decryptInputFileIfNeeded( + TrinoInputFile inputFile, + DeleteFile deleteFile, + Optional encryptionManager) + { + if (encryptionManager.isEmpty() || deleteFile.keyMetadata() == null) { + return inputFile; + } + ByteBuffer keyMetadata = deleteFile.keyMetadata(); + InputFile encryptedInputFile = new ForwardingInputFile(inputFile); + InputFile decryptedInputFile = encryptionManager.get().decrypt(EncryptedFiles.encryptedInput(encryptedInputFile, keyMetadata)); + return new EncryptedTrinoInputFile(inputFile, decryptedInputFile); } private static boolean isDeletionVector(DeleteFile deleteFile) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeleteFile.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeleteFile.java index 6357e58a2327..e56c4a25bd49 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeleteFile.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/DeleteFile.java @@ -15,9 +15,11 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.trino.plugin.iceberg.IcebergSplit.ParquetFileDecryptionData; import org.apache.iceberg.FileContent; import org.apache.iceberg.FileFormat; import org.apache.iceberg.types.Conversions; +import org.apache.iceberg.util.ByteBuffers; import java.nio.ByteBuffer; import java.util.List; @@ -28,6 +30,7 @@ import static io.airlift.slice.SizeOf.SIZE_OF_INT; import static io.airlift.slice.SizeOf.estimatedSizeOf; import static io.airlift.slice.SizeOf.instanceSize; +import static io.airlift.slice.SizeOf.sizeOf; import static java.util.Objects.requireNonNull; import static java.util.Objects.requireNonNullElse; import static org.apache.iceberg.MetadataColumns.DELETE_FILE_POS; @@ -38,6 +41,8 @@ public record DeleteFile( FileFormat format, long recordCount, long fileSizeInBytes, + Optional encryptionKeyMetadata, + Optional parquetFileDecryptionData, List equalityFieldIds, OptionalLong rowPositionLowerBound, OptionalLong rowPositionUpperBound, @@ -47,19 +52,20 @@ public record DeleteFile( { private static final long INSTANCE_SIZE = instanceSize(DeleteFile.class); - public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) + public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile, Optional parquetFileDecryptionData) { ByteBuffer lowerBoundPosition = requireNonNullElse(deleteFile.lowerBounds(), ImmutableMap.of()).get(DELETE_FILE_POS.fieldId()); ByteBuffer upperBoundPosition = requireNonNullElse(deleteFile.upperBounds(), ImmutableMap.of()).get(DELETE_FILE_POS.fieldId()); OptionalLong rowPositionLowerBound = lowerBoundPosition == null ? OptionalLong.empty() : OptionalLong.of(Conversions.fromByteBuffer(DELETE_FILE_POS.type(), lowerBoundPosition)); - OptionalLong rowPositionUpperBound = upperBoundPosition == null ? OptionalLong.empty() : OptionalLong.of(Conversions.fromByteBuffer(DELETE_FILE_POS.type(), upperBoundPosition)); OptionalLong contentOffset = deleteFile.contentOffset() == null ? OptionalLong.empty() : OptionalLong.of(deleteFile.contentOffset()); Optional contentSizeInBytes = Optional.ofNullable(deleteFile.contentSizeInBytes()).map(Math::toIntExact); + Optional encryptionKeyMetadata = Optional.ofNullable(deleteFile.keyMetadata()) + .map(ByteBuffers::toByteArray); return new DeleteFile( deleteFile.content(), @@ -67,6 +73,8 @@ public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) deleteFile.format(), deleteFile.recordCount(), deleteFile.fileSizeInBytes(), + encryptionKeyMetadata, + parquetFileDecryptionData, Optional.ofNullable(deleteFile.equalityFieldIds()).orElseGet(ImmutableList::of), rowPositionLowerBound, rowPositionUpperBound, @@ -80,6 +88,8 @@ public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) requireNonNull(content, "content is null"); requireNonNull(path, "path is null"); requireNonNull(format, "format is null"); + requireNonNull(encryptionKeyMetadata, "encryptionKeyMetadata is null"); + requireNonNull(parquetFileDecryptionData, "parquetFileDecryptionData is null"); equalityFieldIds = ImmutableList.copyOf(requireNonNull(equalityFieldIds, "equalityFieldIds is null")); requireNonNull(rowPositionLowerBound, "rowPositionLowerBound is null"); requireNonNull(rowPositionUpperBound, "rowPositionUpperBound is null"); @@ -99,6 +109,8 @@ public long retainedSizeInBytes() { return INSTANCE_SIZE + estimatedSizeOf(path) + + encryptionKeyMetadata.map(value -> sizeOf(value)).orElse(0L) + + parquetFileDecryptionData.map(ParquetFileDecryptionData::getRetainedSizeInBytes).orElse(0L) + estimatedSizeOf(equalityFieldIds, _ -> SIZE_OF_INT); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/PositionDeleteWriter.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/PositionDeleteWriter.java index b2623e43bbab..4fd29f49cd10 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/PositionDeleteWriter.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/delete/PositionDeleteWriter.java @@ -30,6 +30,7 @@ import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.SortOrder; +import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.LocationProvider; import java.util.Map; @@ -60,7 +61,8 @@ public PositionDeleteWriter( TrinoFileSystem fileSystem, ConnectorSession session, IcebergFileFormat fileFormat, - Map storageProperties) + Map storageProperties, + Optional encryptionManager) { this.dataFilePath = requireNonNull(dataFilePath, "dataFilePath is null"); this.dataFilePathBlock = nativeValueToBlock(VARCHAR, utf8Slice(dataFilePath)); @@ -74,7 +76,7 @@ public PositionDeleteWriter( this.outputPath = partition .map(partitionData -> locationProvider.newDataLocation(partitionSpec, partitionData, fileName)) .orElseGet(() -> locationProvider.newDataLocation(fileName)); - this.writer = fileWriterFactory.createPositionDeleteWriter(fileSystem, Location.of(outputPath), session, fileFormat, storageProperties); + this.writer = fileWriterFactory.createPositionDeleteWriter(fileSystem, Location.of(outputPath), session, fileFormat, storageProperties, encryptionManager); } public CommitTaskData write(DeletionVector rowsToDelete) @@ -82,10 +84,13 @@ public CommitTaskData write(DeletionVector rowsToDelete) writeDeletes(rowsToDelete); writer.commit(); + long fileSizeInBytes = writer.getWrittenBytes(); + Optional encryptionKeyMetadata = writer.getEncryptionKeyMetadata(); + return new CommitTaskData( outputPath, fileFormat, - writer.getWrittenBytes(), + fileSizeInBytes, new MetricsWrapper(writer.getFileMetrics().metrics()), PartitionSpecParser.toJson(partitionSpec), partition.map(PartitionData::toJson), @@ -93,6 +98,7 @@ public CommitTaskData write(DeletionVector rowsToDelete) Optional.of(dataFilePath), writer.getFileMetrics().splitOffsets(), SortOrder.unsorted().orderId(), + encryptionKeyMetadata, Optional.empty()); } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/IcebergEncryptionManagerFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/IcebergEncryptionManagerFactory.java new file mode 100644 index 000000000000..bba0825fbfa8 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/IcebergEncryptionManagerFactory.java @@ -0,0 +1,110 @@ +/* + * Licensed 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 io.trino.plugin.iceberg.encryption; + +import com.google.common.collect.ImmutableMap; +import com.google.inject.Inject; +import io.trino.plugin.iceberg.IcebergConfig; +import io.trino.spi.TrinoException; +import org.apache.iceberg.CatalogProperties; +import org.apache.iceberg.TableMetadata; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.EncryptionUtil; +import org.apache.iceberg.encryption.KeyManagementClient; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; + +import java.util.HashMap; +import java.util.Map; +import java.util.Optional; + +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; +import static java.util.Objects.requireNonNull; + +public class IcebergEncryptionManagerFactory +{ + private final Optional kmsImpl; + private final Map kmsProperties; + + private volatile KeyManagementClient catalogKmsClient; + + @Inject + public IcebergEncryptionManagerFactory(IcebergConfig config) + { + // TODO: Once Iceberg commit https://github.com/apache/iceberg/commit/8c2ca1d084fca37671ba8b38d59ea3f5a187b147 is available in the version we use, + // switch to CatalogProperties.ENCRYPTION_KMS_TYPE instead of mapping to encryption.kms-impl. + this.kmsImpl = config.getEncryptionKmsType() + .map(IcebergConfig.EncryptionKmsType::getKmsClientClassName); + this.kmsProperties = parseKmsProperties(config.getEncryptionKmsProperties()); + } + + public EncryptionManager createEncryptionManager(TableMetadata metadata) + { + requireNonNull(metadata, "metadata is null"); + return createEncryptionManager(metadata.properties()); + } + + public EncryptionManager createEncryptionManager(Map tableProperties) + { + requireNonNull(tableProperties, "tableProperties is null"); + if (!tableProperties.containsKey(TableProperties.ENCRYPTION_TABLE_KEY)) { + return PlaintextEncryptionManager.instance(); + } + + KeyManagementClient client = getOrCreateKmsClient(); + return EncryptionUtil.createEncryptionManager(tableProperties, client); + } + + private KeyManagementClient getOrCreateKmsClient() + { + String configuredKmsImpl = kmsImpl.orElse(null); + if (configuredKmsImpl == null || configuredKmsImpl.isEmpty()) { + throw new TrinoException(NOT_SUPPORTED, "Iceberg table encryption requires iceberg.encryption.kms-type catalog property"); + } + + if (catalogKmsClient != null) { + return catalogKmsClient; + } + + synchronized (this) { + if (catalogKmsClient != null) { + return catalogKmsClient; + } + Map properties = new HashMap<>(kmsProperties); + // Iceberg 1.10.x does not support encryption.kms-type yet, so set the KMS impl explicitly. + properties.remove(CatalogProperties.ENCRYPTION_KMS_TYPE); + properties.put(CatalogProperties.ENCRYPTION_KMS_IMPL, configuredKmsImpl); + catalogKmsClient = EncryptionUtil.createKmsClient(properties); + return catalogKmsClient; + } + } + + private static Map parseKmsProperties(Iterable kmsProperties) + { + Map properties = new HashMap<>(); + for (String property : kmsProperties) { + int delimiter = property.indexOf('='); + if (delimiter <= 0 || delimiter == property.length() - 1) { + throw new IllegalArgumentException("Invalid iceberg.encryption.kms-properties entry: " + property); + } + String key = property.substring(0, delimiter).trim(); + String value = property.substring(delimiter + 1).trim(); + if (key.isEmpty() || value.isEmpty()) { + throw new IllegalArgumentException("Invalid iceberg.encryption.kms-properties entry: " + property); + } + properties.put(key, value); + } + return ImmutableMap.copyOf(properties); + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInput.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInput.java new file mode 100644 index 000000000000..f8402420361a --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInput.java @@ -0,0 +1,87 @@ +/* + * Licensed 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 io.trino.plugin.iceberg.fileio; + +import io.trino.filesystem.TrinoInput; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.SeekableInputStream; + +import java.io.EOFException; +import java.io.IOException; + +import static java.lang.Math.min; +import static java.util.Objects.checkFromIndexSize; +import static java.util.Objects.requireNonNull; + +public class EncryptedTrinoInput + implements TrinoInput +{ + private final InputFile inputFile; + private final SeekableInputStream inputStream; + private final long length; + private boolean closed; + + public EncryptedTrinoInput(InputFile inputFile) + { + this.inputFile = requireNonNull(inputFile, "inputFile is null"); + this.inputStream = inputFile.newStream(); + this.length = inputFile.getLength(); + } + + @Override + public void readFully(long position, byte[] buffer, int bufferOffset, int bufferLength) + throws IOException + { + ensureOpen(); + checkFromIndexSize(bufferOffset, bufferLength, buffer.length); + inputStream.seek(position); + int remaining = bufferLength; + int offset = bufferOffset; + while (remaining > 0) { + int read = inputStream.read(buffer, offset, remaining); + if (read < 0) { + throw new EOFException("Reached end of encrypted input " + inputFile.location()); + } + remaining -= read; + offset += read; + } + } + + @Override + public int readTail(byte[] buffer, int bufferOffset, int bufferLength) + throws IOException + { + ensureOpen(); + checkFromIndexSize(bufferOffset, bufferLength, buffer.length); + int readSize = (int) min(length, bufferLength); + readFully(length - readSize, buffer, bufferOffset, readSize); + return readSize; + } + + @Override + public void close() + throws IOException + { + closed = true; + inputStream.close(); + } + + private void ensureOpen() + throws IOException + { + if (closed) { + throw new IOException("Input closed: " + inputFile.location()); + } + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInputFile.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInputFile.java new file mode 100644 index 000000000000..a1d687df9ef2 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInputFile.java @@ -0,0 +1,81 @@ +/* + * Licensed 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 io.trino.plugin.iceberg.fileio; + +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoInput; +import io.trino.filesystem.TrinoInputFile; +import io.trino.filesystem.TrinoInputStream; +import org.apache.iceberg.io.InputFile; + +import java.io.IOException; +import java.time.Instant; + +import static java.util.Objects.requireNonNull; + +public class EncryptedTrinoInputFile + implements TrinoInputFile +{ + private final TrinoInputFile delegate; + private final InputFile decryptedInputFile; + private final long length; + + public EncryptedTrinoInputFile(TrinoInputFile delegate, InputFile decryptedInputFile) + { + this.delegate = requireNonNull(delegate, "delegate is null"); + this.decryptedInputFile = requireNonNull(decryptedInputFile, "decryptedInputFile is null"); + this.length = decryptedInputFile.getLength(); + } + + @Override + public TrinoInput newInput() + throws IOException + { + return new EncryptedTrinoInput(decryptedInputFile); + } + + @Override + public TrinoInputStream newStream() + throws IOException + { + return new EncryptedTrinoInputStream(decryptedInputFile.newStream()); + } + + @Override + public long length() + throws IOException + { + return length; + } + + @Override + public Instant lastModified() + throws IOException + { + return delegate.lastModified(); + } + + @Override + public boolean exists() + throws IOException + { + return delegate.exists(); + } + + @Override + public Location location() + { + return delegate.location(); + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInputStream.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInputStream.java new file mode 100644 index 000000000000..62c0a7a867d3 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoInputStream.java @@ -0,0 +1,67 @@ +/* + * Licensed 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 io.trino.plugin.iceberg.fileio; + +import io.trino.filesystem.TrinoInputStream; +import org.apache.iceberg.io.SeekableInputStream; + +import java.io.IOException; + +import static java.util.Objects.requireNonNull; + +public class EncryptedTrinoInputStream + extends TrinoInputStream +{ + private final SeekableInputStream input; + + public EncryptedTrinoInputStream(SeekableInputStream input) + { + this.input = requireNonNull(input, "input is null"); + } + + @Override + public long getPosition() + throws IOException + { + return input.getPos(); + } + + @Override + public void seek(long position) + throws IOException + { + input.seek(position); + } + + @Override + public int read() + throws IOException + { + return input.read(); + } + + @Override + public int read(byte[] buffer, int offset, int length) + throws IOException + { + return input.read(buffer, offset, length); + } + + @Override + public void close() + throws IOException + { + input.close(); + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoOutputFile.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoOutputFile.java new file mode 100644 index 000000000000..67d367ddc365 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/fileio/EncryptedTrinoOutputFile.java @@ -0,0 +1,61 @@ +/* + * Licensed 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 io.trino.plugin.iceberg.fileio; + +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoOutputFile; +import io.trino.memory.context.AggregatedMemoryContext; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.PositionOutputStream; + +import java.io.IOException; +import java.io.OutputStream; + +import static java.util.Objects.requireNonNull; + +public class EncryptedTrinoOutputFile + implements TrinoOutputFile +{ + private final Location location; + private final OutputFile encryptingOutputFile; + + public EncryptedTrinoOutputFile(Location location, OutputFile encryptingOutputFile) + { + this.location = requireNonNull(location, "location is null"); + this.encryptingOutputFile = requireNonNull(encryptingOutputFile, "encryptingOutputFile is null"); + } + + @Override + public OutputStream create(AggregatedMemoryContext memoryContext) + throws IOException + { + PositionOutputStream outputStream = encryptingOutputFile.create(); + return outputStream; + } + + @Override + public void createOrOverwrite(byte[] data) + throws IOException + { + try (OutputStream outputStream = encryptingOutputFile.createOrOverwrite()) { + outputStream.write(data); + } + } + + @Override + public Location location() + { + return location; + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunction.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunction.java index 408f6d4d1933..4a54cf4a18fa 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunction.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunction.java @@ -158,6 +158,7 @@ public TableFunctionAnalysis analyze(ConnectorSession session, ConnectorTransact schemaTableName, SchemaParser.toJson(tableSchema), columnHandles, + icebergTable.properties(), Optional.ofNullable(icebergTable.properties().get(TableProperties.DEFAULT_NAME_MAPPING)), startSnapshotId, endSnapshotId)) diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunctionHandle.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunctionHandle.java index 97354093476c..7fd9604a7355 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunctionHandle.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunctionHandle.java @@ -14,11 +14,13 @@ package io.trino.plugin.iceberg.functions.tablechanges; import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import io.trino.plugin.iceberg.IcebergColumnHandle; import io.trino.spi.connector.SchemaTableName; import io.trino.spi.function.table.ConnectorTableFunctionHandle; import java.util.List; +import java.util.Map; import java.util.Optional; import static java.util.Objects.requireNonNull; @@ -27,6 +29,7 @@ public record TableChangesFunctionHandle( SchemaTableName schemaTableName, String tableSchemaJson, List columns, + Map storageProperties, Optional nameMappingJson, long startSnapshotId, long endSnapshotId) implements ConnectorTableFunctionHandle @@ -36,6 +39,7 @@ public record TableChangesFunctionHandle( requireNonNull(schemaTableName, "schemaTableName is null"); requireNonNull(tableSchemaJson, "tableSchemaJson is null"); columns = ImmutableList.copyOf(requireNonNull(columns, "columns is null")); + storageProperties = ImmutableMap.copyOf(storageProperties); requireNonNull(nameMappingJson, "nameMappingJson is null"); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunctionProcessor.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunctionProcessor.java index c1db3c3d09f2..e795b864c067 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunctionProcessor.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesFunctionProcessor.java @@ -32,6 +32,7 @@ import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.mapping.NameMappingParser; import java.io.IOException; @@ -118,6 +119,7 @@ else if (column.getId() == DATA_CHANGE_ORDINAL_ID) { } } + Optional encryptionManager = icebergPageSourceProvider.encryptionManager(functionHandle.storageProperties()); this.pageSource = icebergPageSourceProvider.createPageSource( session, functionHandle.columns(), @@ -126,8 +128,8 @@ else if (column.getId() == DATA_CHANGE_ORDINAL_ID) { PartitionData.fromJson(split.partitionDataJson(), partitionColumnTypes), ImmutableList.of(), DynamicFilter.EMPTY, - TupleDomain.all(), - TupleDomain.all(), + TupleDomain.all(), + TupleDomain.all(), split.path(), split.start(), split.length(), @@ -138,7 +140,10 @@ else if (column.getId() == DATA_CHANGE_ORDINAL_ID) { getFileIoProperties(tableCredentials), 0, OptionalLong.empty(), - functionHandle.nameMappingJson().map(NameMappingParser::fromJson)); + functionHandle.nameMappingJson().map(NameMappingParser::fromJson), + split.encryptionKeyMetadata(), + split.parquetFileDecryptionData(), + encryptionManager); this.delegateColumnMap = delegateColumnMap; this.changeTypeIndex = changeTypeIndex; diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesSplit.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesSplit.java index b665d58eedcc..861380905931 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesSplit.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesSplit.java @@ -15,9 +15,12 @@ import io.airlift.slice.SizeOf; import io.trino.plugin.iceberg.IcebergFileFormat; +import io.trino.plugin.iceberg.IcebergSplit.ParquetFileDecryptionData; import io.trino.spi.SplitWeight; import io.trino.spi.connector.ConnectorSplit; +import java.util.Optional; + import static com.google.common.base.MoreObjects.toStringHelper; import static io.airlift.slice.SizeOf.estimatedSizeOf; import static java.util.Objects.requireNonNull; @@ -33,6 +36,8 @@ public record TableChangesSplit( long fileSize, long fileRecordCount, IcebergFileFormat fileFormat, + Optional encryptionKeyMetadata, + Optional parquetFileDecryptionData, String partitionSpecJson, String partitionDataJson, SplitWeight splitWeight) implements ConnectorSplit @@ -44,6 +49,8 @@ public record TableChangesSplit( requireNonNull(changeType, "changeType is null"); requireNonNull(path, "path is null"); requireNonNull(fileFormat, "fileFormat is null"); + requireNonNull(encryptionKeyMetadata, "encryptionKeyMetadata is null"); + requireNonNull(parquetFileDecryptionData, "parquetFileDecryptionData is null"); requireNonNull(partitionSpecJson, "partitionSpecJson is null"); requireNonNull(partitionDataJson, "partitionDataJson is null"); requireNonNull(splitWeight, "splitWeight is null"); @@ -62,7 +69,9 @@ public long getRetainedSizeInBytes() + estimatedSizeOf(path) + estimatedSizeOf(partitionSpecJson) + estimatedSizeOf(partitionDataJson) - + splitWeight.getRetainedSizeInBytes(); + + splitWeight.getRetainedSizeInBytes() + + encryptionKeyMetadata.map(SizeOf::sizeOf).orElse(0L) + + parquetFileDecryptionData.map(ParquetFileDecryptionData::getRetainedSizeInBytes).orElse(0L); } @Override diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesSplitSource.java index 719bce4ecbea..65b046b78950 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/functions/tablechanges/TableChangesSplitSource.java @@ -15,6 +15,8 @@ import com.google.common.io.Closer; import io.trino.plugin.iceberg.IcebergFileFormat; +import io.trino.plugin.iceberg.IcebergSplit.ParquetFileDecryptionData; +import io.trino.plugin.iceberg.IcebergSplitSource; import io.trino.plugin.iceberg.PartitionData; import io.trino.spi.SplitWeight; import io.trino.spi.TrinoException; @@ -23,19 +25,26 @@ import io.trino.spi.type.DateTimeEncoding; import org.apache.iceberg.AddedRowsScanTask; import org.apache.iceberg.ChangelogScanTask; +import org.apache.iceberg.ContentFile; import org.apache.iceberg.DeletedDataFileScanTask; import org.apache.iceberg.IncrementalChangelogScan; import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.SplittableScanTask; import org.apache.iceberg.Table; +import org.apache.iceberg.encryption.EncryptedFiles; +import org.apache.iceberg.encryption.EncryptedInputFile; +import org.apache.iceberg.encryption.EncryptionManager; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.util.ByteBuffers; import java.io.IOException; import java.io.UncheckedIOException; import java.util.ArrayList; import java.util.Iterator; import java.util.List; +import java.util.Optional; import java.util.concurrent.CompletableFuture; import static com.google.common.collect.Iterators.singletonIterator; @@ -44,11 +53,14 @@ import static java.util.Collections.emptyIterator; import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.apache.iceberg.FileFormat.PARQUET; public class TableChangesSplitSource implements ConnectorSplitSource { private final Table icebergTable; + private final FileIO fileIo; + private final Optional encryptionManager; private final IncrementalChangelogScan tableScan; private final long targetSplitSize; private final Closer closer = Closer.create(); @@ -59,9 +71,12 @@ public class TableChangesSplitSource public TableChangesSplitSource( Table icebergTable, + Optional encryptionManager, IncrementalChangelogScan tableScan) { this.icebergTable = requireNonNull(icebergTable, "table is null"); + this.fileIo = requireNonNull(icebergTable.io(), "fileIo is null"); + this.encryptionManager = requireNonNull(encryptionManager, "encryptionManager is null"); this.tableScan = requireNonNull(tableScan, "tableScan is null"); this.targetSplitSize = tableScan.targetSplitSize(); } @@ -151,6 +166,8 @@ private TableChangesSplit toSplit(AddedRowsScanTask task) task.file().fileSizeInBytes(), task.file().recordCount(), IcebergFileFormat.fromIceberg(task.file().format()), + Optional.ofNullable(task.file().keyMetadata()).map(ByteBuffers::toByteArray), + parquetFileDecryptionData(task.file()), PartitionSpecParser.toJson(task.spec()), PartitionData.toJson(task.file().partition()), SplitWeight.standard()); @@ -169,8 +186,22 @@ private TableChangesSplit toSplit(DeletedDataFileScanTask task) task.file().fileSizeInBytes(), task.file().recordCount(), IcebergFileFormat.fromIceberg(task.file().format()), + Optional.ofNullable(task.file().keyMetadata()).map(ByteBuffers::toByteArray), + parquetFileDecryptionData(task.file()), PartitionSpecParser.toJson(task.spec()), PartitionData.toJson(task.file().partition()), SplitWeight.standard()); } + + private Optional parquetFileDecryptionData(ContentFile file) + { + if (file.format() != PARQUET || file.keyMetadata() == null || encryptionManager.isEmpty()) { + return Optional.empty(); + } + + EncryptedInputFile encryptedInputFile = EncryptedFiles.encryptedInput( + fileIo.newInputFile(file.location()), + file.keyMetadata()); + return IcebergSplitSource.parquetFileDecryptionData(encryptedInputFile, encryptionManager.orElseThrow()); + } } diff --git a/plugin/trino-iceberg/src/main/java/org/apache/iceberg/snowflake/SnowflakeIcebergTableOperations.java b/plugin/trino-iceberg/src/main/java/org/apache/iceberg/snowflake/SnowflakeIcebergTableOperations.java index 912e8e84e4f4..d959b67a604b 100644 --- a/plugin/trino-iceberg/src/main/java/org/apache/iceberg/snowflake/SnowflakeIcebergTableOperations.java +++ b/plugin/trino-iceberg/src/main/java/org/apache/iceberg/snowflake/SnowflakeIcebergTableOperations.java @@ -15,6 +15,7 @@ import io.trino.plugin.iceberg.catalog.AbstractIcebergTableOperations; import io.trino.plugin.iceberg.catalog.snowflake.TrinoSnowflakeCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.TableMetadata; @@ -37,6 +38,7 @@ public class SnowflakeIcebergTableOperations public SnowflakeIcebergTableOperations( TrinoSnowflakeCatalog trinoSnowflakeCatalog, FileIO fileIo, + IcebergEncryptionManagerFactory encryptionManagerFactory, ConnectorSession session, String snowflakeDatabase, String database, @@ -44,7 +46,7 @@ public SnowflakeIcebergTableOperations( Optional owner, Optional location) { - super(fileIo, session, database, table, owner, location); + super(fileIo, encryptionManagerFactory, session, database, table, owner, location); TableIdentifier tableIdentifier = TableIdentifier.of(Namespace.of(snowflakeDatabase, database), table); this.icebergSnowflakeTableOperations = requireNonNull((SnowflakeTableOperations) trinoSnowflakeCatalog.getSnowflakeCatalog().newTableOps(tableIdentifier), "snowflakeTableOperations is null"); } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java index c73b52632632..3f554dd2215c 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergTestUtils.java @@ -46,6 +46,7 @@ import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.file.FileMetastoreTableOperationsProvider; import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.plugin.iceberg.fileio.ForwardingInputFile; import io.trino.spi.block.Block; @@ -230,8 +231,22 @@ public static BaseTable loadTable(String tableName, String catalogName, String schemaName) { - IcebergTableOperationsProvider tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY); - TrinoCatalog catalog = getTrinoCatalog(metastore, fileSystemFactory, catalogName); + IcebergEncryptionManagerFactory encryptionManagerFactory = new IcebergEncryptionManagerFactory(new IcebergConfig()); + return loadTable(tableName, metastore, fileSystemFactory, catalogName, schemaName, encryptionManagerFactory); + } + + public static BaseTable loadTable(String tableName, + HiveMetastore metastore, + TrinoFileSystemFactory fileSystemFactory, + String catalogName, + String schemaName, + IcebergEncryptionManagerFactory encryptionManagerFactory) + { + IcebergTableOperationsProvider tableOperationsProvider = new FileMetastoreTableOperationsProvider( + fileSystemFactory, + FILE_IO_FACTORY, + encryptionManagerFactory); + TrinoCatalog catalog = getTrinoCatalog(metastore, fileSystemFactory, catalogName, encryptionManagerFactory); return loadIcebergTable(catalog, tableOperationsProvider, SESSION, new SchemaTableName(schemaName, tableName)); } @@ -240,7 +255,20 @@ public static TrinoCatalog getTrinoCatalog( TrinoFileSystemFactory fileSystemFactory, String catalogName) { - IcebergTableOperationsProvider tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY); + IcebergEncryptionManagerFactory encryptionManagerFactory = new IcebergEncryptionManagerFactory(new IcebergConfig()); + return getTrinoCatalog(metastore, fileSystemFactory, catalogName, encryptionManagerFactory); + } + + public static TrinoCatalog getTrinoCatalog( + HiveMetastore metastore, + TrinoFileSystemFactory fileSystemFactory, + String catalogName, + IcebergEncryptionManagerFactory encryptionManagerFactory) + { + IcebergTableOperationsProvider tableOperationsProvider = new FileMetastoreTableOperationsProvider( + fileSystemFactory, + FILE_IO_FACTORY, + encryptionManagerFactory); CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); return new TrinoHiveCatalog( new CatalogName(catalogName), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestFileBasedConflictDetection.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestFileBasedConflictDetection.java index 2ce99b1a2a59..5bebd70b10b9 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestFileBasedConflictDetection.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestFileBasedConflictDetection.java @@ -212,10 +212,10 @@ void testConflictDetectionOnEvolvedTable() {"partitionValues":[40]} """; CommitTaskData commitTaskData1 = new CommitTaskData("test_location/data/new.parquet", IcebergFileFormat.PARQUET, 0, new MetricsWrapper(new Metrics()), PartitionSpecParser.toJson(currentPartitionSpec), - Optional.of(partitionDataJson), DATA, Optional.empty(), Optional.empty(), SortOrder.unsorted().orderId(), Optional.empty()); + Optional.of(partitionDataJson), DATA, Optional.empty(), Optional.empty(), SortOrder.unsorted().orderId(), Optional.empty(), Optional.empty()); // Remove file from version with previous partition specification CommitTaskData commitTaskData2 = new CommitTaskData("test_location/data/old.parquet", IcebergFileFormat.PARQUET, 0, new MetricsWrapper(new Metrics()), PartitionSpecParser.toJson(previousPartitionSpec), - Optional.of(partitionDataJson), POSITION_DELETES, Optional.empty(), Optional.empty(), SortOrder.unsorted().orderId(), Optional.empty()); + Optional.of(partitionDataJson), POSITION_DELETES, Optional.empty(), Optional.empty(), SortOrder.unsorted().orderId(), Optional.empty(), Optional.empty()); TupleDomain icebergColumnHandleTupleDomain = extractTupleDomainsFromCommitTasks(getIcebergTableHandle(currentPartitionSpec), icebergTable, List.of(commitTaskData1, commitTaskData2), null); assertThat(icebergColumnHandleTupleDomain.getDomains().orElseThrow()).isEmpty(); @@ -236,6 +236,7 @@ private static List getCommitTaskDataForUpdate(PartitionSpec par Optional.empty(), Optional.empty(), SortOrder.unsorted().orderId(), + Optional.empty(), Optional.empty()); CommitTaskData commitTaskData2 = new CommitTaskData( "test_location/data/old.parquet", @@ -248,6 +249,7 @@ private static List getCommitTaskDataForUpdate(PartitionSpec par Optional.empty(), Optional.empty(), SortOrder.unsorted().orderId(), + Optional.empty(), Optional.empty()); return List.of(commitTaskData1, commitTaskData2); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java index d6d0d0b8ffd1..d658f370a74e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergConfig.java @@ -40,10 +40,10 @@ import static java.util.concurrent.TimeUnit.HOURS; import static java.util.concurrent.TimeUnit.SECONDS; -public class TestIcebergConfig +final class TestIcebergConfig { @Test - public void testDefaults() + void testDefaults() { assertRecordedDefaults(recordDefaults(IcebergConfig.class) .setFileFormat(PARQUET) @@ -87,11 +87,13 @@ public void testDefaults() .setObjectStoreLayoutEnabled(false) .setMetadataParallelism(8) .setBucketExecutionEnabled(true) - .setFileBasedConflictDetectionEnabled(true)); + .setFileBasedConflictDetectionEnabled(true) + .setEncryptionKmsType(null) + .setEncryptionKmsProperties(ImmutableList.of())); } @Test - public void testExplicitPropertyMappings() + void testExplicitPropertyMappings() { Map properties = ImmutableMap.builder() .put("iceberg.file-format", "ORC") @@ -135,6 +137,8 @@ public void testExplicitPropertyMappings() .put("iceberg.metadata.parallelism", "10") .put("iceberg.bucket-execution", "false") .put("iceberg.file-based-conflict-detection", "false") + .put("iceberg.encryption.kms-type", "aws") + .put("iceberg.encryption.kms-properties", "client.factory=example.ClientFactory,key.id=test-key") .buildOrThrow(); IcebergConfig expected = new IcebergConfig() @@ -179,13 +183,15 @@ public void testExplicitPropertyMappings() .setObjectStoreLayoutEnabled(true) .setMetadataParallelism(10) .setBucketExecutionEnabled(false) - .setFileBasedConflictDetectionEnabled(false); + .setFileBasedConflictDetectionEnabled(false) + .setEncryptionKmsType(IcebergConfig.EncryptionKmsType.AWS) + .setEncryptionKmsProperties(ImmutableList.of("client.factory=example.ClientFactory", "key.id=test-key")); assertFullMapping(properties, expected); } @Test - public void testValidation() + void testValidation() { assertFailsValidation( new IcebergConfig() diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java index d9844a87d377..df2a93867299 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergMergeAppend.java @@ -21,6 +21,7 @@ import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.file.FileMetastoreTableOperationsProvider; import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.SchemaTableName; import io.trino.testing.AbstractTestQueryFramework; @@ -51,7 +52,8 @@ protected QueryRunner createQueryRunner() HiveMetastore metastore = getHiveMetastore(queryRunner); CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); TrinoFileSystemFactory fileSystemFactory = getFileSystemFactory(queryRunner); - tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY); + IcebergEncryptionManagerFactory encryptionManagerFactory = new IcebergEncryptionManagerFactory(new IcebergConfig()); + tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, encryptionManagerFactory); trinoCatalog = new TrinoHiveCatalog( new CatalogName("catalog"), cachingHiveMetastore, diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java index 9bf6a3d782fd..67116b4ccf64 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergNodeLocalDynamicSplitPruning.java @@ -34,6 +34,7 @@ import io.trino.plugin.hive.orc.OrcWriterConfig; import io.trino.plugin.hive.parquet.ParquetReaderConfig; import io.trino.plugin.hive.parquet.ParquetWriterConfig; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.Page; import io.trino.spi.SplitWeight; import io.trino.spi.block.BlockBuilder; @@ -146,6 +147,8 @@ public void testDynamicSplitPruningOnUnpartitionedTable() -1, // invalid; normally known ORC, PartitionSpec.unpartitioned().specId(), + Optional.empty(), + Optional.empty(), PartitionData.toJson(new PartitionData(new Object[] {})), ImmutableList.of(), SplitWeight.standard(), @@ -208,6 +211,8 @@ public void testDynamicSplitPruningOnUnpartitionedTable() -1, // invalid; normally known ORC, PartitionSpec.unpartitioned().specId(), + Optional.empty(), + Optional.empty(), PartitionData.toJson(new PartitionData(new Object[] {})), ImmutableList.of(), SplitWeight.standard(), @@ -318,7 +323,9 @@ public void testDynamicSplitPruningWithExplicitPartitionFilter() -1, // invalid; normally known ORC, partitionSpec.specId(), - PartitionData.toJson(new PartitionData(new Object[] {dateColumnValue})), + Optional.empty(), + Optional.empty(), + PartitionData.toJson(new PartitionData(new Object[] {(int) dateColumnValue})), ImmutableList.of(), SplitWeight.standard(), TupleDomain.all(), @@ -471,7 +478,9 @@ public void testDynamicSplitPruningWithExplicitPartitionFilterPartitionEvolution -1, // invalid; normally known ORC, partitionSpec.specId(), - PartitionData.toJson(new PartitionData(new Object[] {yearColumnValue})), + Optional.empty(), + Optional.empty(), + PartitionData.toJson(new PartitionData(new Object[] {(int) yearColumnValue})), ImmutableList.of(), SplitWeight.standard(), TupleDomain.all(), @@ -582,7 +591,8 @@ private static ConnectorPageSource createTestingPageSource( stats, ORC_READER_CONFIG, PARQUET_READER_CONFIG, - TESTING_TYPE_MANAGER); + TESTING_TYPE_MANAGER, + new IcebergEncryptionManagerFactory(icebergConfig)); return factory.createPageSourceProvider().createPageSource( transaction, getSession(icebergConfig), diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java index 729968550903..156ae53ad112 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergOrcMetricsCollection.java @@ -23,6 +23,7 @@ import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.file.FileMetastoreTableOperationsProvider; import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.SchemaTableName; @@ -85,7 +86,8 @@ protected QueryRunner createQueryRunner() queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", ImmutableMap.of("iceberg.file-format", "ORC")); TrinoFileSystemFactory fileSystemFactory = getFileSystemFactory(queryRunner); - tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY); + IcebergEncryptionManagerFactory encryptionManagerFactory = new IcebergEncryptionManagerFactory(new IcebergConfig()); + tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, encryptionManagerFactory); HiveMetastore metastore = getHiveMetastore(queryRunner); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergPageSourceProvider.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergPageSourceProvider.java index 9ee830ee5d0d..2e5c4d910e87 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergPageSourceProvider.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergPageSourceProvider.java @@ -25,6 +25,7 @@ import io.trino.plugin.hive.parquet.ParquetReaderConfig; import io.trino.plugin.hive.parquet.ParquetWriterConfig; import io.trino.plugin.iceberg.delete.DeleteFile; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.Page; import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; @@ -104,6 +105,8 @@ void testMemoryTrackingWithEqualityDeletes(@TempDir Path tempDir) FileFormat.PARQUET, 1, // recordCount deleteInputFile.length(), + Optional.empty(), + Optional.empty(), ImmutableList.of(regionkeyIdentity.getId()), // equalityFieldIds OptionalLong.empty(), OptionalLong.empty(), @@ -145,6 +148,9 @@ void testMemoryTrackingWithEqualityDeletes(@TempDir Path tempDir) ImmutableMap.of(), 0L, // dataSequenceNumber OptionalLong.empty(), + Optional.empty(), + Optional.empty(), + Optional.empty(), Optional.empty())) { // Memory should still be 0 before reading any pages (lazy loading) assertThat(provider.getMemoryUsage()).isEqualTo(0); @@ -192,6 +198,7 @@ private static IcebergPageSourceProvider createPageSourceProvider() new FileFormatDataSourceStats(), ORC_READER_CONFIG.toOrcReaderOptions(), PARQUET_READER_CONFIG.toParquetReaderOptions(), - TESTING_TYPE_MANAGER); + TESTING_TYPE_MANAGER, + new IcebergEncryptionManagerFactory(new IcebergConfig())); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java index 2271d22eb42a..b7f295ed6b90 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergSplitSource.java @@ -29,6 +29,7 @@ import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.file.FileMetastoreTableOperationsProvider; import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.SplitWeight; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ColumnHandle; @@ -120,6 +121,7 @@ protected QueryRunner createQueryRunner() this.fileSystemFactory = getFileSystemFactory(queryRunner); CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); + IcebergEncryptionManagerFactory encryptionManagerFactory = new IcebergEncryptionManagerFactory(new IcebergConfig()); this.catalog = new TrinoHiveCatalog( new CatalogName("hive"), cachingHiveMetastore, @@ -127,7 +129,7 @@ protected QueryRunner createQueryRunner() fileSystemFactory, FILE_IO_FACTORY, TESTING_TYPE_MANAGER, - new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY), + new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, encryptionManagerFactory), false, false, false, @@ -153,6 +155,7 @@ public void testIncompleteDynamicFilterTimeout() SESSION, tableHandle, nationTable, + Optional.empty(), nationTable.newScan(), Optional.empty(), new DynamicFilter() @@ -410,6 +413,7 @@ private IcebergSplit generateSplit(Table nationTable, IcebergTableHandle tableHa SESSION, tableHandle, nationTable, + Optional.empty(), nationTable.newScan(), Optional.empty(), dynamicFilter, diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableEncryption.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableEncryption.java new file mode 100644 index 000000000000..d1b2de10dc4f --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableEncryption.java @@ -0,0 +1,148 @@ +/* + * Licensed 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 io.trino.plugin.iceberg; + +import com.google.common.collect.ImmutableList; +import io.trino.Session; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.HiveMetastore; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.TestingPropertyAwareAwsClientFactory; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.TestTable; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileScanTask; +import org.junit.jupiter.api.Test; + +import java.util.List; +import java.util.stream.StreamSupport; + +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; +import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; +import static io.trino.plugin.iceberg.IcebergTestUtils.getHiveMetastore; +import static io.trino.plugin.iceberg.IcebergTestUtils.loadTable; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestIcebergTableEncryption + extends AbstractTestQueryFramework +{ + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return IcebergQueryRunner.builder() + .addIcebergProperty("iceberg.encryption.kms-type", "aws") + .addIcebergProperty( + "iceberg.encryption.kms-properties", + "client.factory=%s,%s=configured".formatted( + TestingPropertyAwareAwsClientFactory.class.getName(), + TestingPropertyAwareAwsClientFactory.REQUIRED_PROPERTY)) + .build(); + } + + @Test + void testEncryptedTableOperations() + { + try (TestTable table = new TestTable( + getQueryRunner()::execute, + "test_encrypted_table_", + "(id INT, data VARCHAR) WITH (" + + "format_version = 3, " + + "encryption_key_id = 'test-key', " + + "encryption_data_key_length = 16)")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a'), (2, 'b'), (3, 'c')", 3); + assertUpdate("UPDATE " + tableName + " SET data = 'bb' WHERE id = 2", 1); + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 1); + assertQuery("SELECT id, data FROM " + tableName + " ORDER BY id", "VALUES (2, 'bb'), (3, 'c')"); + + assertEncryptedFilesHaveKeyMetadata(tableName); + } + } + + @Test + void testEncryptedOrcWriterValidation() + { + Session session = Session.builder(getSession()) + .setCatalogSessionProperty(ICEBERG_CATALOG, "orc_writer_validate_percentage", "100") + .build(); + + try (TestTable table = new TestTable( + sql -> getQueryRunner().execute(session, sql), + "test_encrypted_table_orc_validate_", + "(id INT, data VARCHAR) WITH (" + + "format = 'ORC', " + + "format_version = 3, " + + "encryption_key_id = 'test-key', " + + "encryption_data_key_length = 16)")) { + String tableName = table.getName(); + assertUpdate(session, "INSERT INTO " + tableName + " VALUES (1, 'a'), (2, 'b')", 2); + assertQuery(session, "SELECT id, data FROM " + tableName + " ORDER BY id", "VALUES (1, 'a'), (2, 'b')"); + } + } + + @Test + void testEncryptedTableChangesFunction() + { + try (TestTable table = new TestTable( + getQueryRunner()::execute, + "test_encrypted_table_changes_", + "(id INT, data VARCHAR) WITH (" + + "format_version = 3, " + + "encryption_key_id = 'test-key', " + + "encryption_data_key_length = 16)")) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a')", 1); + long snapshotAfterFirstInsert = ((Number) computeScalar("SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1")).longValue(); + + assertUpdate("INSERT INTO " + tableName + " VALUES (2, 'b')", 1); + long snapshotAfterSecondInsert = ((Number) computeScalar("SELECT snapshot_id FROM \"" + tableName + "$snapshots\" ORDER BY committed_at DESC LIMIT 1")).longValue(); + + assertQuery( + "SELECT id, data, _change_type, _change_version_id FROM TABLE(system.table_changes(CURRENT_SCHEMA, '" + tableName + "', " + snapshotAfterFirstInsert + ", " + snapshotAfterSecondInsert + "))", + "VALUES (2, 'b', 'insert', " + snapshotAfterSecondInsert + ")"); + } + } + + private void assertEncryptedFilesHaveKeyMetadata(String tableName) + { + HiveMetastore metastore = getHiveMetastore(getQueryRunner()); + TrinoFileSystemFactory fileSystemFactory = getFileSystemFactory(getQueryRunner()); + IcebergConfig icebergConfig = new IcebergConfig() + .setEncryptionKmsType(IcebergConfig.EncryptionKmsType.AWS) + .setEncryptionKmsProperties(ImmutableList.of( + "client.factory=" + TestingPropertyAwareAwsClientFactory.class.getName(), + TestingPropertyAwareAwsClientFactory.REQUIRED_PROPERTY + "=configured")); + IcebergEncryptionManagerFactory encryptionManagerFactory = new IcebergEncryptionManagerFactory(icebergConfig); + BaseTable icebergTable = loadTable( + tableName, + metastore, + fileSystemFactory, + ICEBERG_CATALOG, + getSession().getSchema().orElseThrow(), + encryptionManagerFactory); + + List scanTasks = StreamSupport.stream(icebergTable.newScan().planFiles().spliterator(), false) + .toList(); + assertThat(scanTasks).isNotEmpty(); + scanTasks.forEach(task -> assertThat(task.file().keyMetadata()).isNotNull()); + + List deleteFiles = scanTasks.stream() + .flatMap(task -> task.deletes().stream()) + .toList(); + deleteFiles.forEach(deleteFile -> assertThat(deleteFile.keyMetadata()).isNotNull()); + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableEncryptionWithLocalStackKms.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableEncryptionWithLocalStackKms.java new file mode 100644 index 000000000000..71bf2c30ced4 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergTableEncryptionWithLocalStackKms.java @@ -0,0 +1,155 @@ +/* + * Licensed 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 io.trino.plugin.iceberg; + +import com.google.common.collect.ImmutableList; +import io.trino.filesystem.TrinoFileSystemFactory; +import io.trino.metastore.HiveMetastore; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.TestingLocalStackAwsClientFactory; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import io.trino.testing.sql.TestTable; +import org.apache.iceberg.BaseTable; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.FileScanTask; +import org.junit.jupiter.api.Test; +import org.testcontainers.containers.GenericContainer; +import org.testcontainers.containers.wait.strategy.Wait; +import org.testcontainers.utility.DockerImageName; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.kms.KmsClient; + +import java.net.URI; +import java.time.Duration; +import java.util.List; +import java.util.stream.StreamSupport; + +import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; +import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; +import static io.trino.plugin.iceberg.IcebergTestUtils.getHiveMetastore; +import static io.trino.plugin.iceberg.IcebergTestUtils.loadTable; +import static java.lang.String.format; +import static org.assertj.core.api.Assertions.assertThat; + +final class TestIcebergTableEncryptionWithLocalStackKms + extends AbstractTestQueryFramework +{ + private static final DockerImageName LOCALSTACK_IMAGE = DockerImageName.parse("localstack/localstack:4.0.3"); + private static final int LOCALSTACK_PORT = 4566; + + private GenericContainer localstack; + private String kmsKeyArn; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + localstack = closeAfterClass(new GenericContainer<>(LOCALSTACK_IMAGE) + .withEnv("SERVICES", "kms") + .withEnv("EAGER_SERVICE_LOADING", "1") + .withEnv("AWS_DEFAULT_REGION", Region.US_EAST_1.id()) + .withExposedPorts(LOCALSTACK_PORT) + .waitingFor(Wait.forHttp("/_localstack/health") + .forPort(LOCALSTACK_PORT) + .forStatusCode(200)) + .withStartupTimeout(Duration.ofMinutes(2))); + localstack.start(); + + URI kmsEndpoint = localstackEndpoint(); + kmsKeyArn = createKmsKey(kmsEndpoint); + + return IcebergQueryRunner.builder() + .addIcebergProperty("iceberg.encryption.kms-type", "aws") + .addIcebergProperty( + "iceberg.encryption.kms-properties", + "client.factory=%s,%s=%s".formatted( + TestingLocalStackAwsClientFactory.class.getName(), + TestingLocalStackAwsClientFactory.KMS_ENDPOINT_PROPERTY, + kmsEndpoint)) + .build(); + } + + @Test + void testEncryptedTableOperationsWithLocalStackKms() + { + String tableDefinition = format( + "(id INT, data VARCHAR) WITH (" + + "format_version = 3, " + + "encryption_key_id = '%s', " + + "encryption_data_key_length = 16)", + kmsKeyArn); + + try (TestTable table = new TestTable( + getQueryRunner()::execute, + "test_encrypted_table_localstack_", + tableDefinition)) { + String tableName = table.getName(); + assertUpdate("INSERT INTO " + tableName + " VALUES (1, 'a'), (2, 'b'), (3, 'c')", 3); + assertUpdate("UPDATE " + tableName + " SET data = 'bb' WHERE id = 2", 1); + assertUpdate("DELETE FROM " + tableName + " WHERE id = 1", 1); + assertQuery("SELECT id, data FROM " + tableName + " ORDER BY id", "VALUES (2, 'bb'), (3, 'c')"); + assertEncryptedFilesHaveKeyMetadata(tableName); + } + } + + private URI localstackEndpoint() + { + return URI.create("http://" + localstack.getHost() + ":" + localstack.getMappedPort(LOCALSTACK_PORT)); + } + + private static String createKmsKey(URI endpoint) + { + try (KmsClient kmsClient = KmsClient.builder() + .endpointOverride(endpoint) + .region(Region.US_EAST_1) + .credentialsProvider(StaticCredentialsProvider.create(AwsBasicCredentials.create("test", "test"))) + .build()) { + return kmsClient.createKey(request -> request.description("Trino Iceberg encryption test key")) + .keyMetadata() + .arn(); + } + } + + private void assertEncryptedFilesHaveKeyMetadata(String tableName) + { + HiveMetastore metastore = getHiveMetastore(getQueryRunner()); + TrinoFileSystemFactory fileSystemFactory = getFileSystemFactory(getQueryRunner()); + IcebergConfig icebergConfig = new IcebergConfig() + .setEncryptionKmsType(IcebergConfig.EncryptionKmsType.AWS) + .setEncryptionKmsProperties(ImmutableList.of( + "client.factory=" + TestingLocalStackAwsClientFactory.class.getName(), + TestingLocalStackAwsClientFactory.KMS_ENDPOINT_PROPERTY + "=" + localstackEndpoint())); + IcebergEncryptionManagerFactory encryptionManagerFactory = new IcebergEncryptionManagerFactory(icebergConfig); + BaseTable icebergTable = loadTable( + tableName, + metastore, + fileSystemFactory, + ICEBERG_CATALOG, + getSession().getSchema().orElseThrow(), + encryptionManagerFactory); + + List scanTasks = StreamSupport.stream(icebergTable.newScan().planFiles().spliterator(), false) + .toList(); + assertThat(scanTasks).isNotEmpty(); + scanTasks.forEach(task -> assertThat(task.file().keyMetadata()).isNotNull()); + + List deleteFiles = scanTasks.stream() + .flatMap(task -> task.deletes().stream()) + .toList(); + deleteFiles.forEach(deleteFile -> assertThat(deleteFile.keyMetadata()).isNotNull()); + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV3.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV3.java index 668ef2c7180f..8c07fe695ec4 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV3.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergV3.java @@ -82,7 +82,7 @@ import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.keycloak.util.JsonSerialization.mapper; -public class TestIcebergV3 +final class TestIcebergV3 extends AbstractTestQueryFramework { private static final List ALL_FILE_FORMATS = List.of("PARQUET", "ORC", "AVRO"); @@ -1240,36 +1240,33 @@ private BaseTable loadTable(String tableName) } @Test - void testV3RejectsEncryptionKeyProperty() + void testV3RequiresKmsClientForEncryptionKeyProperty() { String tableName = "test_v3_encryption_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (id INTEGER) WITH (format = 'ORC', format_version = 3)"); assertUpdate("INSERT INTO " + tableName + " VALUES 1", 1); - // Set encryption.key-id property via Iceberg API BaseTable icebergTable = loadTable(tableName); icebergTable.updateProperties() .set("encryption.key-id", "test_key") .commit(); - assertQueryFails( - "SELECT * FROM " + tableName, - ".*Iceberg table encryption is not supported.*"); + assertThatThrownBy(() -> getQueryRunner().execute("SELECT * FROM " + tableName)) + .hasMessageContaining("Iceberg table encryption requires iceberg.encryption.kms-type catalog property"); - // Also verify INSERT fails with encryption key set - assertQueryFails( - "INSERT INTO " + tableName + " VALUES 2", - ".*Iceberg table encryption is not supported.*"); + // Also verify INSERT fails with encryption key set and no KMS client. + assertThatThrownBy(() -> getQueryRunner().execute("INSERT INTO " + tableName + " VALUES 2")) + .hasMessageContaining("Iceberg table encryption requires iceberg.encryption.kms-type catalog property"); - // Clean up by removing the property first - icebergTable.updateProperties() + // The key cannot be removed without configuring KMS client either. + assertThatThrownBy(() -> icebergTable.updateProperties() .remove("encryption.key-id") - .commit(); - assertUpdate("DROP TABLE " + tableName); + .commit()) + .hasMessageContaining("Iceberg table encryption requires iceberg.encryption.kms-type catalog property"); } @Test - void testV3RejectsEncryptionKeysInMetadata() + void testV3AllowsEncryptionKeysInMetadata() throws Exception { String temp = "tmp_v3_encryption_src_" + randomNameSuffix(); @@ -1301,9 +1298,7 @@ void testV3RejectsEncryptionKeysInMetadata() assertUpdate("CALL system.register_table(CURRENT_SCHEMA, '%s', '%s')" .formatted(registered, hadoopTableLocation)); - assertQueryFails( - "SELECT * FROM " + registered, - ".*Iceberg table encryption is not supported.*"); + assertQuery("SELECT * FROM " + registered, "VALUES 1"); // Use unregister_table instead of DROP TABLE because DROP TABLE triggers the same validation error assertUpdate("CALL system.unregister_table(CURRENT_SCHEMA, '%s')".formatted(registered)); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestAbstractIcebergTableOperations.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestAbstractIcebergTableOperations.java index 9987ebb3f0ca..dbb7425aaa72 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestAbstractIcebergTableOperations.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestAbstractIcebergTableOperations.java @@ -17,6 +17,8 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.local.LocalFileSystemFactory; import io.trino.metastore.HiveMetastore; +import io.trino.plugin.iceberg.IcebergConfig; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIo; import org.apache.iceberg.io.InputFile; import org.junit.jupiter.api.Test; @@ -56,6 +58,7 @@ public InputFile newInputFile(String path) throw new UncheckedIOException(new IOException()); } }, + new IcebergEncryptionManagerFactory(new IcebergConfig()), createPerTransactionCache(metastore, 1000), SESSION, "test-database", diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java index 9c9329608e7d..692672ac3ba7 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/file/TestTrinoHiveCatalogWithFileMetastore.java @@ -27,6 +27,7 @@ import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.SchemaTableName; @@ -113,7 +114,7 @@ protected TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations) fileSystemFactory, FILE_IO_FACTORY, TESTING_TYPE_MANAGER, - new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY), + new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, new IcebergEncryptionManagerFactory(new IcebergConfig())), useUniqueTableLocations, false, false, diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java index 3c0c4c25ea12..3fa5013926fe 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/glue/TestTrinoGlueCatalog.java @@ -25,6 +25,7 @@ import io.trino.plugin.iceberg.TableStatisticsWriter; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.NodeVersion; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; @@ -105,7 +106,8 @@ private TrinoCatalog createGlueTrinoCatalog(boolean useUniqueTableLocations, boo TESTING_TYPE_MANAGER, catalogConfig, new GlueMetastoreStats(), - glueClient), + glueClient, + new IcebergEncryptionManagerFactory(new IcebergConfig())), "test", new StatsRecordingGlueClient(glueClient, new GlueMetastoreStats()), useSystemSecurity, @@ -261,7 +263,8 @@ public void testDefaultLocation() TESTING_TYPE_MANAGER, catalogConfig, new GlueMetastoreStats(), - glueClient), + glueClient, + new IcebergEncryptionManagerFactory(new IcebergConfig())), "test", new StatsRecordingGlueClient(glueClient, new GlueMetastoreStats()), false, diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithHiveMetastore.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithHiveMetastore.java index 6e56a7498c4e..06ca9d7f145d 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithHiveMetastore.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithHiveMetastore.java @@ -36,9 +36,11 @@ import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory; +import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergSchemaProperties; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.CatalogSchemaTableName; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; @@ -181,7 +183,8 @@ public ThriftMetastore createMetastore(Optional identity) return thriftMetastore; } }, - new IcebergHiveCatalogConfig()), + new IcebergHiveCatalogConfig(), + new IcebergEncryptionManagerFactory(new IcebergConfig())), useUniqueTableLocations, false, false, diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestTrinoNessieCatalog.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestTrinoNessieCatalog.java index 2395b293cba6..15ab86188496 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestTrinoNessieCatalog.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/nessie/TestTrinoNessieCatalog.java @@ -17,11 +17,13 @@ import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.plugin.iceberg.CommitTaskData; +import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergMetadata; import io.trino.plugin.iceberg.TableStatisticsWriter; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.plugin.iceberg.containers.NessieContainer; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.NodeVersion; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMetadata; @@ -121,7 +123,7 @@ protected TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations) TESTING_TYPE_MANAGER, fileSystemFactory, FILE_IO_FACTORY, - new IcebergNessieTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, nessieClient), + new IcebergNessieTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, nessieClient, new IcebergEncryptionManagerFactory(new IcebergConfig())), nessieClient, tmpDirectory.toAbsolutePath().toString(), useUniqueTableLocations); @@ -146,7 +148,7 @@ public void testDefaultLocation() TESTING_TYPE_MANAGER, fileSystemFactory, FILE_IO_FACTORY, - new IcebergNessieTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, nessieClient), + new IcebergNessieTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, nessieClient, new IcebergEncryptionManagerFactory(new IcebergConfig())), nessieClient, icebergNessieCatalogConfig.getDefaultWarehouseDir(), false); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestIcebergEncryptionManagerFactory.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestIcebergEncryptionManagerFactory.java new file mode 100644 index 000000000000..f7927e067cac --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestIcebergEncryptionManagerFactory.java @@ -0,0 +1,70 @@ +/* + * Licensed 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 io.trino.plugin.iceberg.encryption; + +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; +import io.trino.plugin.iceberg.IcebergConfig; +import org.apache.iceberg.TableProperties; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; +import org.junit.jupiter.api.Test; + +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; + +final class TestIcebergEncryptionManagerFactory +{ + @Test + void testReturnsPlaintextManagerWhenEncryptionDisabled() + { + IcebergEncryptionManagerFactory factory = new IcebergEncryptionManagerFactory(new IcebergConfig()); + assertThat(factory.createEncryptionManager(ImmutableMap.of())) + .isSameAs(PlaintextEncryptionManager.instance()); + } + + @Test + void testRequiresCatalogKmsConfigurationWhenEncryptionEnabled() + { + IcebergEncryptionManagerFactory factory = new IcebergEncryptionManagerFactory(new IcebergConfig()); + + assertThatThrownBy(() -> factory.createEncryptionManager(ImmutableMap.of( + TableProperties.ENCRYPTION_TABLE_KEY, "key", + "encryption.kms-impl", "test.kms.Client"))) + .hasMessageContaining("Iceberg table encryption requires iceberg.encryption.kms-type catalog property"); + } + + @Test + void testCatalogKmsClientUsesKmsProperties() + { + IcebergEncryptionManagerFactory factory = new IcebergEncryptionManagerFactory(new IcebergConfig() + .setEncryptionKmsType(IcebergConfig.EncryptionKmsType.AWS) + .setEncryptionKmsProperties(ImmutableList.of( + "client.factory=" + TestingPropertyAwareAwsClientFactory.class.getName(), + TestingPropertyAwareAwsClientFactory.REQUIRED_PROPERTY + "=catalog-value"))); + + assertThat(factory.createEncryptionManager(ImmutableMap.of(TableProperties.ENCRYPTION_TABLE_KEY, "key"))) + .isNotNull(); + } + + @Test + void testRejectsInvalidKmsPropertiesEntry() + { + assertThatThrownBy(() -> new IcebergEncryptionManagerFactory( + new IcebergConfig() + .setEncryptionKmsType(IcebergConfig.EncryptionKmsType.AWS) + .setEncryptionKmsProperties(ImmutableList.of("invalid")))) + .isInstanceOf(IllegalArgumentException.class) + .hasMessageContaining("Invalid iceberg.encryption.kms-properties entry"); + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestingLocalStackAwsClientFactory.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestingLocalStackAwsClientFactory.java new file mode 100644 index 000000000000..6fcfd8a2add7 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestingLocalStackAwsClientFactory.java @@ -0,0 +1,86 @@ +/* + * Licensed 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 io.trino.plugin.iceberg.encryption; + +import org.apache.iceberg.aws.AwsClientFactory; +import software.amazon.awssdk.auth.credentials.AwsBasicCredentials; +import software.amazon.awssdk.auth.credentials.StaticCredentialsProvider; +import software.amazon.awssdk.regions.Region; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.kms.KmsClient; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Client; + +import java.net.URI; +import java.util.Map; + +import static java.util.Objects.requireNonNull; + +public class TestingLocalStackAwsClientFactory + implements AwsClientFactory +{ + public static final String KMS_ENDPOINT_PROPERTY = "localstack.kms.endpoint"; + + private URI kmsEndpoint; + + @Override + public void initialize(Map properties) + { + String endpoint = properties.get(KMS_ENDPOINT_PROPERTY); + if (endpoint == null || endpoint.isEmpty()) { + throw new IllegalArgumentException("Missing required property: " + KMS_ENDPOINT_PROPERTY); + } + kmsEndpoint = URI.create(endpoint); + } + + @Override + public S3Client s3() + { + throw unsupportedClient("s3"); + } + + @Override + public S3AsyncClient s3Async() + { + throw unsupportedClient("s3Async"); + } + + @Override + public GlueClient glue() + { + throw unsupportedClient("glue"); + } + + @Override + public KmsClient kms() + { + return KmsClient.builder() + .endpointOverride(requireNonNull(kmsEndpoint, "Factory must be initialized before use")) + .region(Region.US_EAST_1) + .credentialsProvider(StaticCredentialsProvider.create(AwsBasicCredentials.create("test", "test"))) + .build(); + } + + @Override + public DynamoDbClient dynamo() + { + throw unsupportedClient("dynamo"); + } + + private static UnsupportedOperationException unsupportedClient(String clientName) + { + return new UnsupportedOperationException("Testing factory supports only kms client, not " + clientName); + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestingPropertyAwareAwsClientFactory.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestingPropertyAwareAwsClientFactory.java new file mode 100644 index 000000000000..d7b06a82d175 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestingPropertyAwareAwsClientFactory.java @@ -0,0 +1,133 @@ +/* + * Licensed 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 io.trino.plugin.iceberg.encryption; + +import org.apache.iceberg.aws.AwsClientFactory; +import software.amazon.awssdk.core.SdkBytes; +import software.amazon.awssdk.services.dynamodb.DynamoDbClient; +import software.amazon.awssdk.services.glue.GlueClient; +import software.amazon.awssdk.services.kms.KmsClient; +import software.amazon.awssdk.services.kms.model.DataKeySpec; +import software.amazon.awssdk.services.kms.model.DecryptRequest; +import software.amazon.awssdk.services.kms.model.DecryptResponse; +import software.amazon.awssdk.services.kms.model.EncryptRequest; +import software.amazon.awssdk.services.kms.model.EncryptResponse; +import software.amazon.awssdk.services.kms.model.GenerateDataKeyRequest; +import software.amazon.awssdk.services.kms.model.GenerateDataKeyResponse; +import software.amazon.awssdk.services.s3.S3AsyncClient; +import software.amazon.awssdk.services.s3.S3Client; + +import java.security.SecureRandom; +import java.util.Map; + +public class TestingPropertyAwareAwsClientFactory + implements AwsClientFactory +{ + public static final String REQUIRED_PROPERTY = "testing.aws.required-property"; + + private KmsClient kmsClient; + + @Override + public void initialize(Map properties) + { + String value = properties.get(REQUIRED_PROPERTY); + if (value == null || value.isEmpty()) { + throw new IllegalArgumentException("Missing required property: " + REQUIRED_PROPERTY); + } + kmsClient = new InMemoryKmsClient(); + } + + @Override + public S3Client s3() + { + throw unsupportedClient("s3"); + } + + @Override + public S3AsyncClient s3Async() + { + throw unsupportedClient("s3Async"); + } + + @Override + public GlueClient glue() + { + throw unsupportedClient("glue"); + } + + @Override + public KmsClient kms() + { + if (kmsClient == null) { + throw new IllegalStateException("Factory must be initialized before use"); + } + return kmsClient; + } + + @Override + public DynamoDbClient dynamo() + { + throw unsupportedClient("dynamo"); + } + + private static UnsupportedOperationException unsupportedClient(String clientName) + { + return new UnsupportedOperationException("Testing factory supports only kms client, not " + clientName); + } + + private static class InMemoryKmsClient + implements KmsClient + { + private final SecureRandom secureRandom = new SecureRandom(); + + @Override + public EncryptResponse encrypt(EncryptRequest request) + { + return EncryptResponse.builder() + .ciphertextBlob(request.plaintext()) + .build(); + } + + @Override + public DecryptResponse decrypt(DecryptRequest request) + { + return DecryptResponse.builder() + .plaintext(request.ciphertextBlob()) + .build(); + } + + @Override + public GenerateDataKeyResponse generateDataKey(GenerateDataKeyRequest request) + { + int keyLength = request.keySpec() == DataKeySpec.AES_128 ? 16 : 32; + byte[] key = new byte[keyLength]; + secureRandom.nextBytes(key); + SdkBytes keyBytes = SdkBytes.fromByteArray(key); + + return GenerateDataKeyResponse.builder() + .plaintext(keyBytes) + .ciphertextBlob(keyBytes) + .build(); + } + + @Override + public String serviceName() + { + return "kms"; + } + + @Override + public void close() {} + } +} diff --git a/plugin/trino-iceberg/src/test/java/org/apache/iceberg/snowflake/TestTrinoSnowflakeCatalog.java b/plugin/trino-iceberg/src/test/java/org/apache/iceberg/snowflake/TestTrinoSnowflakeCatalog.java index 6bc50bf0d5c2..452dd1873053 100644 --- a/plugin/trino-iceberg/src/test/java/org/apache/iceberg/snowflake/TestTrinoSnowflakeCatalog.java +++ b/plugin/trino-iceberg/src/test/java/org/apache/iceberg/snowflake/TestTrinoSnowflakeCatalog.java @@ -23,6 +23,7 @@ import io.trino.metastore.TableInfo; import io.trino.plugin.iceberg.ColumnIdentity; import io.trino.plugin.iceberg.CommitTaskData; +import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergMetadata; import io.trino.plugin.iceberg.TableStatisticsWriter; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; @@ -32,6 +33,7 @@ import io.trino.plugin.iceberg.catalog.snowflake.SnowflakeIcebergTableOperationsProvider; import io.trino.plugin.iceberg.catalog.snowflake.TestingSnowflakeServer; import io.trino.plugin.iceberg.catalog.snowflake.TrinoSnowflakeCatalog; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.spi.NodeVersion; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMetadata; @@ -183,7 +185,11 @@ protected TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations) SnowflakeCatalog snowflakeCatalog = new SnowflakeCatalog(); snowflakeCatalog.initialize(catalogName.toString(), snowflakeClient, catalogFileIOFactory, properties); - IcebergTableOperationsProvider tableOperationsProvider = new SnowflakeIcebergTableOperationsProvider(s3FileSystemFactory, FILE_IO_FACTORY, CATALOG_CONFIG); + IcebergTableOperationsProvider tableOperationsProvider = new SnowflakeIcebergTableOperationsProvider( + s3FileSystemFactory, + FILE_IO_FACTORY, + new IcebergEncryptionManagerFactory(new IcebergConfig()), + CATALOG_CONFIG); return new TrinoSnowflakeCatalog( snowflakeCatalog, diff --git a/plugin/trino-lakehouse/src/main/java/io/trino/plugin/lakehouse/LakehouseIcebergModule.java b/plugin/trino-lakehouse/src/main/java/io/trino/plugin/lakehouse/LakehouseIcebergModule.java index 487dd690bbb4..2803b67793b0 100644 --- a/plugin/trino-lakehouse/src/main/java/io/trino/plugin/lakehouse/LakehouseIcebergModule.java +++ b/plugin/trino-lakehouse/src/main/java/io/trino/plugin/lakehouse/LakehouseIcebergModule.java @@ -42,6 +42,7 @@ import io.trino.plugin.iceberg.catalog.hms.IcebergHiveMetastoreCatalogModule; import io.trino.plugin.iceberg.delete.DefaultDeletionVectorWriter; import io.trino.plugin.iceberg.delete.DeletionVectorWriter; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; @@ -70,6 +71,7 @@ protected void setup(Binder binder) binder.bind(IcebergFileWriterFactory.class).in(Scopes.SINGLETON); binder.bind(TableStatisticsReader.class).in(Scopes.SINGLETON); binder.bind(TableStatisticsWriter.class).in(Scopes.SINGLETON); + binder.bind(IcebergEncryptionManagerFactory.class).in(Scopes.SINGLETON); binder.bind(IcebergFileSystemFactory.class).to(DefaultIcebergFileSystemFactory.class).in(Scopes.SINGLETON); newOptionalBinder(binder, Key.get(HiveMetastoreFactory.class, RawHiveMetastoreFactory.class));