diff --git a/docs/src/main/sphinx/connector/iceberg.md b/docs/src/main/sphinx/connector/iceberg.md index 98a453b8cf4f..b41974486b87 100644 --- a/docs/src/main/sphinx/connector/iceberg.md +++ b/docs/src/main/sphinx/connector/iceberg.md @@ -259,6 +259,18 @@ implementation is used: - Enable bucket-aware execution. This allows the engine to use physical bucketing information to optimize queries by reducing data exchanges. - `true` +* - `iceberg.encryption.kms-type` + - Key Management Service type for + [Iceberg table encryption](https://iceberg.apache.org/docs/nightly/encryption/). + Possible values are `AWS` and `GCP`. Required to read encrypted tables. + Writing to encrypted tables is not supported. + - +* - `iceberg.encryption.plaintext-files-allowed-for-encrypted-tables` + - Allow reading unencrypted files in tables with encryption enabled. When set + to `false`, an error is raised if a file with encryption key metadata is not + actually encrypted. The equivalent catalog session property is + `plaintext_files_allowed_for_encrypted_tables`. + - `false` ::: (iceberg-fte-support)= diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/crypto/FileDecryptionProperties.java b/lib/trino-parquet/src/main/java/io/trino/parquet/crypto/FileDecryptionProperties.java index 574ca1252829..d241ba40d397 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/crypto/FileDecryptionProperties.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/crypto/FileDecryptionProperties.java @@ -22,12 +22,18 @@ public class FileDecryptionProperties private final DecryptionKeyRetriever keyRetriever; private final Optional aadPrefix; private final boolean checkFooterIntegrity; + private final boolean plaintextFilesAllowed; - private FileDecryptionProperties(DecryptionKeyRetriever keyRetriever, Optional aadPrefix, boolean checkFooterIntegrity) + private FileDecryptionProperties( + DecryptionKeyRetriever keyRetriever, + Optional aadPrefix, + boolean checkFooterIntegrity, + boolean plaintextFilesAllowed) { this.keyRetriever = requireNonNull(keyRetriever, "keyRetriever is null"); this.aadPrefix = requireNonNull(aadPrefix, "aadPrefix is null"); this.checkFooterIntegrity = checkFooterIntegrity; + this.plaintextFilesAllowed = plaintextFilesAllowed; } public static Builder builder() @@ -50,11 +56,17 @@ public boolean isCheckFooterIntegrity() return checkFooterIntegrity; } + public boolean isPlaintextFilesAllowed() + { + return plaintextFilesAllowed; + } + public static class Builder { private DecryptionKeyRetriever keyRetriever; private Optional aadPrefix = Optional.empty(); private boolean checkFooterIntegrity = true; + private boolean plaintextFilesAllowed; public Builder withKeyRetriever(DecryptionKeyRetriever keyRetriever) { @@ -74,9 +86,15 @@ public Builder withCheckFooterIntegrity(boolean checkFooterIntegrity) return this; } + public Builder withPlaintextFilesAllowed() + { + this.plaintextFilesAllowed = true; + return this; + } + public FileDecryptionProperties build() { - return new FileDecryptionProperties(keyRetriever, aadPrefix, checkFooterIntegrity); + return new FileDecryptionProperties(keyRetriever, aadPrefix, checkFooterIntegrity, plaintextFilesAllowed); } } } diff --git a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/MetadataReader.java b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/MetadataReader.java index 1c1463caeb21..7a51182ac18c 100644 --- a/lib/trino-parquet/src/main/java/io/trino/parquet/reader/MetadataReader.java +++ b/lib/trino-parquet/src/main/java/io/trino/parquet/reader/MetadataReader.java @@ -27,6 +27,7 @@ import io.trino.parquet.crypto.AesGcmEncryptor; import io.trino.parquet.crypto.FileDecryptionContext; import io.trino.parquet.crypto.FileDecryptionProperties; +import io.trino.parquet.crypto.ParquetCryptoException; import io.trino.parquet.metadata.FileMetadata; import io.trino.parquet.metadata.ParquetMetadata; import org.apache.parquet.CorruptStatistics; @@ -139,12 +140,19 @@ public static ParquetMetadata readFooter(ParquetDataSource dataSource, DataSize } FileMetaData fileMetaData = readFileMetaData(metadataStream, footerDecryptor, aad); - if (!encryptedFooterMode && fileDecryptionProperties.isPresent() && fileMetaData.isSetEncryption_algorithm()) { - // footer is not encrypted, but some columns might be encrypted - decryptionContext = Optional.of(new FileDecryptionContext(dataSource.getId(), fileDecryptionProperties.get(), fileMetaData.getEncryption_algorithm(), Optional.ofNullable(fileMetaData.getFooter_signing_key_metadata()))); - if (fileDecryptionProperties.get().isCheckFooterIntegrity()) { - // verify footer integrity - verifyFooterIntegrity(dataSource, metadataStream, decryptionContext.get(), metadataLength); + if (!encryptedFooterMode && fileDecryptionProperties.isPresent()) { + if (!fileMetaData.isSetEncryption_algorithm()) { + // Plaintext file — detect files that were not encrypted by mistake + if (!fileDecryptionProperties.get().isPlaintextFilesAllowed()) { + throw new ParquetCryptoException("Applying decryptor on plaintext file: %s", dataSource.getId()); + } + } + else { + // Footer is not encrypted, but some columns might be encrypted + decryptionContext = Optional.of(new FileDecryptionContext(dataSource.getId(), fileDecryptionProperties.get(), fileMetaData.getEncryption_algorithm(), Optional.ofNullable(fileMetaData.getFooter_signing_key_metadata()))); + if (fileDecryptionProperties.get().isCheckFooterIntegrity()) { + verifyFooterIntegrity(dataSource, metadataStream, decryptionContext.get(), metadataLength); + } } } @@ -228,10 +236,10 @@ private static org.apache.parquet.column.statistics.Statistics tryReadOldUtf8 return org.apache.parquet.column.statistics.Statistics .getBuilderForReading(columnStatistics.type()) - .withMin(min) - .withMax(max) - .withNumNulls(!columnStatistics.isNumNullsSet() && statistics.isSetNull_count() ? statistics.getNull_count() : columnStatistics.getNumNulls()) - .build(); + .withMin(min) + .withMax(max) + .withNumNulls(!columnStatistics.isNumNullsSet() && statistics.isSetNull_count() ? statistics.getNull_count() : columnStatistics.getNumNulls()) + .build(); } private static boolean isAscii(byte b) diff --git a/plugin/trino-iceberg/pom.xml b/plugin/trino-iceberg/pom.xml index fc059bde7243..cf8d66f4f55d 100644 --- a/plugin/trino-iceberg/pom.xml +++ b/plugin/trino-iceberg/pom.xml @@ -406,6 +406,22 @@ + + com.google.cloud + google-cloud-kms + runtime + + + com.google.guava + listenablefuture + + + javax.annotation + javax.annotation-api + + + + io.airlift http-client @@ -807,6 +823,16 @@ + + maven-jar-plugin + + + + test-jar + + + + 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 b3a9ad0a1256..85c0c7ffc44e 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 @@ -202,6 +202,7 @@ import org.apache.iceberg.UpdateProperties; import org.apache.iceberg.UpdateSchema; import org.apache.iceberg.UpdateStatistics; +import org.apache.iceberg.encryption.PlaintextEncryptionManager; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; @@ -769,11 +770,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) @@ -1477,6 +1473,7 @@ public ConnectorOutputTableHandle beginCreateTable(ConnectorSession session, Con throw new TrinoException(INVALID_TABLE_PROPERTY, format("The provided location '%s' does not match the existing table location '%s'", providedTableLocation.get(), icebergTable.location())); } validateNotModifyingOldSnapshot(table, icebergTable); + validateNotEncryptedForWrite(icebergTable); tableLocation = icebergTable.location(); List existingPartitionFields = getAllPartitionFields(icebergTable); transaction = newCreateTableTransaction(catalog, tableMetadata, session, replace, tableLocation, allowedExtraProperties, existingPartitionFields); @@ -1606,6 +1603,7 @@ public ConnectorInsertTableHandle beginInsert(ConnectorSession session, Connecto validateNotModifyingOldSnapshot(table, icebergTable); validateTableForTrino(icebergTable, getCurrentSnapshotId(icebergTable)); + validateNotEncryptedForWrite(icebergTable); beginTransaction(icebergTable); @@ -2093,6 +2091,7 @@ private BeginTableExecuteResult executeAddFiles(ConnectorSession session, IcebergTableE { IcebergAddFilesHandle addFilesHandle = (IcebergAddFilesHandle) executeHandle.procedureHandle(); Table table = catalog.loadTable(session, executeHandle.schemaTableName()); + validateNotEncryptedForWrite(table); TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), table.io().properties()); long addedDataFiles = addFiles( session, @@ -2655,6 +2655,7 @@ public Map executeAddFilesFromTable(ConnectorSession session, Iceb { IcebergAddFilesFromTableHandle addFilesHandle = (IcebergAddFilesFromTableHandle) executeHandle.procedureHandle(); Table table = catalog.loadTable(session, executeHandle.schemaTableName()); + validateNotEncryptedForWrite(table); TrinoFileSystem fileSystem = fileSystemFactory.create(session.getIdentity(), table.io().properties()); long addedDataFiles = addFilesFromTable( session, @@ -3549,6 +3550,7 @@ public ConnectorMergeTableHandle beginMerge(ConnectorSession session, ConnectorT Table icebergTable = catalog.loadTable(session, table.getSchemaTableName()); validateNotModifyingOldSnapshot(table, icebergTable); + validateNotEncryptedForWrite(icebergTable); beginTransaction(icebergTable); @@ -3579,6 +3581,13 @@ private static void validateNotModifyingOldSnapshot(IcebergTableHandle table, Ta } } + private static void validateNotEncryptedForWrite(Table table) + { + if (!(table.encryption() instanceof PlaintextEncryptionManager)) { + throw new TrinoException(NOT_SUPPORTED, "Writing to encrypted Iceberg tables is not supported"); + } + } + private void finishWrite(ConnectorSession session, IcebergTableHandle table, Collection fragments) { Table icebergTable = transaction.table(); @@ -4259,6 +4268,7 @@ public ConnectorInsertTableHandle beginRefreshMaterializedView( checkState(fromSnapshotForRefresh.isEmpty(), "From Snapshot must be empty at the start of MV refresh operation."); IcebergTableHandle table = (IcebergTableHandle) tableHandle; Table icebergTable = catalog.loadTable(session, table.getSchemaTableName()); + validateNotEncryptedForWrite(icebergTable); beginTransaction(icebergTable); Optional dependencies = Optional.ofNullable(icebergTable.currentSnapshot()) 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..15bae4e0e16f 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,9 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.EncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.plugin.iceberg.functions.IcebergFunctionProvider; import io.trino.plugin.iceberg.functions.tablechanges.TableChangesFunctionProcessorProviderFactory; @@ -100,6 +103,10 @@ public void configure(Binder binder) configBinder(binder).bindConfig(ParquetReaderConfig.class); configBinder(binder).bindConfig(ParquetWriterConfig.class); + configBinder(binder).bindConfig(IcebergEncryptionConfig.class); + newOptionalBinder(binder, EncryptionManagerFactory.class) + .setDefault().to(DefaultEncryptionManagerFactory.class).in(Scopes.SINGLETON); + binder.bind(ForwardingFileIoFactory.class).in(Scopes.SINGLETON); binder.bind(TableStatisticsReader.class).in(Scopes.SINGLETON); binder.bind(TableStatisticsWriter.class).in(Scopes.SINGLETON); 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 27668c50c6f5..d079d07fc248 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,12 @@ 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.EncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.plugin.iceberg.fileio.ForwardingInputFile; import io.trino.plugin.iceberg.system.files.FilesTablePageSource; @@ -98,6 +102,9 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; import org.apache.iceberg.avro.AvroSchemaUtil; +import org.apache.iceberg.encryption.EncryptingFileIO; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.InputFile; import org.apache.iceberg.mapping.MappedField; import org.apache.iceberg.mapping.MappedFields; @@ -109,6 +116,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; @@ -161,6 +169,7 @@ import static io.trino.plugin.iceberg.IcebergMetadataColumn.FILE_MODIFIED_TIME; import static io.trino.plugin.iceberg.IcebergMetadataColumn.FILE_PATH; import static io.trino.plugin.iceberg.IcebergMetadataColumn.PARTITION; +import static io.trino.plugin.iceberg.IcebergSessionProperties.arePlaintextFilesAllowedForEncryptedTables; import static io.trino.plugin.iceberg.IcebergSessionProperties.getOrcLazyReadSmallRanges; import static io.trino.plugin.iceberg.IcebergSessionProperties.getOrcMaxBufferSize; import static io.trino.plugin.iceberg.IcebergSessionProperties.getOrcMaxMergeDistance; @@ -208,6 +217,7 @@ import static org.apache.iceberg.FileContent.EQUALITY_DELETES; import static org.apache.iceberg.FileContent.POSITION_DELETES; import static org.apache.iceberg.MetadataColumns.ROW_POSITION; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; import static org.joda.time.DateTimeZone.UTC; public class IcebergPageSourceProvider @@ -227,6 +237,7 @@ public class IcebergPageSourceProvider private final OrcReaderOptions orcReaderOptions; private final ParquetReaderOptions parquetReaderOptions; private final TypeManager typeManager; + private final EncryptionManagerFactory 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, + EncryptionManagerFactory 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); } @@ -259,10 +272,15 @@ public ConnectorPageSource createPageSource( DynamicFilter dynamicFilter) { if (connectorSplit instanceof FilesTableSplit filesTableSplit) { + FileIO fileIO = fileIoFactory.create(fileSystemFactory.create(session.getIdentity(), getFileIoProperties(connectorTableCredentials))); + if (filesTableSplit.encryptionKeyId().isPresent()) { + EncryptionManager encryptionManager = encryptionManagerFactory.create( + ImmutableMap.of(ENCRYPTION_TABLE_KEY, filesTableSplit.encryptionKeyId().get())); + fileIO = EncryptingFileIO.combine(fileIO, encryptionManager); + } return new FilesTablePageSource( typeManager, - fileSystemFactory.create(session.getIdentity(), getFileIoProperties(connectorTableCredentials)), - fileIoFactory, + fileIO, columns.stream().map(SystemColumnHandle.class::cast).map(SystemColumnHandle::columnName).collect(toImmutableList()), filesTableSplit); } @@ -298,7 +316,8 @@ public ConnectorPageSource createPageSource( getFileIoProperties(connectorTableCredentials), split.dataSequenceNumber(), split.fileFirstRowId(), - tableHandle.getNameMappingJson().map(NameMappingParser::fromJson)); + tableHandle.getNameMappingJson().map(NameMappingParser::fromJson), + split.parquetFileDecryptionData()); } public ConnectorPageSource createPageSource( @@ -320,7 +339,8 @@ public ConnectorPageSource createPageSource( Map fileIoProperties, long dataSequenceNumber, OptionalLong fileFirstRowId, - Optional nameMapping) + Optional nameMapping, + Optional parquetFileDecryptionData) { Map> partitionKeys = getPartitionKeys(partitionData, partitionSpec); TupleDomain effectivePredicate = getUnenforcedPredicate( @@ -361,6 +381,7 @@ public ConnectorPageSource createPageSource( .filter(not(icebergColumns::contains)) .forEach(requiredColumns::add); + Optional parquetFileDecryptionProperties = createParquetFileDecryptionProperties(parquetFileDecryptionData, arePlaintextFilesAllowedForEncryptedTables(session)); ReaderPageSourceWithRowPositions readerPageSourceWithRowPositions = createDataPageSource( session, inputFile, @@ -377,7 +398,8 @@ public ConnectorPageSource createPageSource( partition, partitionKeys, dataSequenceNumber, - fileFirstRowId); + fileFirstRowId, + parquetFileDecryptionProperties); ConnectorPageSource pageSource = readerPageSourceWithRowPositions.pageSource(); @@ -543,7 +565,8 @@ public ConnectorPageSource openDeleteFile( "", ImmutableMap.of(), 0, - OptionalLong.empty()) + OptionalLong.empty(), + createParquetFileDecryptionProperties(delete.parquetFileDecryptionData(), arePlaintextFilesAllowedForEncryptedTables(session))) .pageSource(); } @@ -563,7 +586,8 @@ private ReaderPageSourceWithRowPositions createDataPageSource( String partition, Map> partitionKeys, long dataSequenceNumber, - OptionalLong fileFirstRowId) + OptionalLong fileFirstRowId, + Optional parquetFileDecryptionProperties) { return switch (fileFormat) { case ORC -> createOrcPageSource( @@ -616,7 +640,8 @@ private ReaderPageSourceWithRowPositions createDataPageSource( partition, partitionKeys, dataSequenceNumber, - fileFirstRowId); + fileFirstRowId, + parquetFileDecryptionProperties); case AVRO -> createAvroPageSource( inputFile, start, @@ -1006,14 +1031,15 @@ private static ReaderPageSourceWithRowPositions createParquetPageSource( String partition, Map> partitionKeys, long dataSequenceNumber, - OptionalLong fileFirstRowId) + OptionalLong fileFirstRowId, + Optional parquetFileDecryptionProperties) { AggregatedMemoryContext memoryContext = newSimpleAggregatedMemoryContext(); ParquetDataSource dataSource = null; try { dataSource = createDataSource(inputFile, OptionalLong.of(fileSize), options, memoryContext, fileFormatDataSourceStats); - ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, options, Optional.empty(), Optional.empty()); + ParquetMetadata parquetMetadata = MetadataReader.readFooter(dataSource, options, Optional.empty(), parquetFileDecryptionProperties); FileMetadata fileMetaData = parquetMetadata.getFileMetaData(); MessageType fileSchema = fileMetaData.getSchema(); if (nameMapping.isPresent() && !ParquetSchemaUtil.hasIds(fileSchema)) { @@ -1172,7 +1198,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); @@ -1212,6 +1238,48 @@ else if (column.isBaseColumn()) { } } + @VisibleForTesting + static Optional createParquetFileDecryptionProperties( + Optional parquetFileDecryptionData, + boolean plaintextFilesAllowed) + { + requireNonNull(parquetFileDecryptionData, "parquetFileDecryptionData is null"); + + if (parquetFileDecryptionData.isEmpty()) { + return Optional.empty(); + } + + ParquetFileDecryptionData decryptionData = parquetFileDecryptionData.get(); + FileDecryptionProperties.Builder builder = FileDecryptionProperties.builder() + .withKeyRetriever(new FixedKeyDecryptionKeyRetriever(decryptionData.fileEncryptionKey())); + builder.withAadPrefix(decryptionData.fileAadPrefix()); + if (plaintextFilesAllowed) { + builder.withPlaintextFilesAllowed(); + } + return Optional.of(builder.build()); + } + + private record FixedKeyDecryptionKeyRetriever(byte[] fileKey) + implements DecryptionKeyRetriever + { + private FixedKeyDecryptionKeyRetriever(byte[] fileKey) + { + this.fileKey = requireNonNull(fileKey, "fileKey is null").clone(); + } + + @Override + public Optional getColumnKey(ColumnPath columnPath, Optional keyMetadata) + { + return Optional.of(fileKey.clone()); + } + + @Override + public Optional getFooterKey(Optional keyMetadata) + { + return Optional.of(fileKey.clone()); + } + } + private static Map createParquetIdToFieldMapping(MessageType fileSchema) { ImmutableMap.Builder builder = ImmutableMap.builder(); 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..ace36e248cf6 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.EncryptionManagerFactory; 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 EncryptionManagerFactory encryptionManagerFactory; @Inject public IcebergPageSourceProviderFactory( @@ -42,7 +44,8 @@ public IcebergPageSourceProviderFactory( FileFormatDataSourceStats fileFormatDataSourceStats, OrcReaderConfig orcReaderConfig, ParquetReaderConfig parquetReaderConfig, - TypeManager typeManager) + TypeManager typeManager, + EncryptionManagerFactory 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/IcebergSessionProperties.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java index 84fca8c296ee..1ae0eec95422 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSessionProperties.java @@ -23,6 +23,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.IcebergEncryptionConfig; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.session.PropertyMetadata; @@ -106,6 +107,7 @@ public final class IcebergSessionProperties private static final String QUERY_PARTITION_FILTER_REQUIRED_SCHEMAS = "query_partition_filter_required_schemas"; private static final String INCREMENTAL_REFRESH_ENABLED = "incremental_refresh_enabled"; public static final String BUCKET_EXECUTION_ENABLED = "bucket_execution_enabled"; + private static final String PLAINTEXT_FILES_ALLOWED_FOR_ENCRYPTED_TABLES = "plaintext_files_allowed_for_encrypted_tables"; private static final String MAX_PARTITIONS_PER_WRITER = "max_partitions_per_writer"; private final List> sessionProperties; @@ -113,6 +115,7 @@ public final class IcebergSessionProperties @Inject public IcebergSessionProperties( IcebergConfig icebergConfig, + IcebergEncryptionConfig encryptionConfig, OrcReaderConfig orcReaderConfig, OrcWriterConfig orcWriterConfig, ParquetReaderConfig parquetReaderConfig, @@ -384,6 +387,11 @@ public IcebergSessionProperties( "Enable bucket-aware execution: use physical bucketing information to optimize queries", icebergConfig.isBucketExecutionEnabled(), false)) + .add(booleanProperty( + PLAINTEXT_FILES_ALLOWED_FOR_ENCRYPTED_TABLES, + "Allow reading unencrypted files in tables with encryption enabled", + encryptionConfig.isPlaintextFilesAllowedForEncryptedTables(), + false)) .add(integerProperty( MAX_PARTITIONS_PER_WRITER, "Maximum number of partitions per writer", @@ -639,4 +647,9 @@ public static int maxPartitionsPerWriter(ConnectorSession session) { return session.getProperty(MAX_PARTITIONS_PER_WRITER, Integer.class); } + + public static boolean arePlaintextFilesAllowedForEncryptedTables(ConnectorSession session) + { + return session.getProperty(PLAINTEXT_FILES_ALLOWED_FOR_ENCRYPTED_TABLES, Boolean.class); + } } 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 bf94f02fd37e..5254f52bfcda 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 @@ -23,7 +23,9 @@ import io.trino.spi.connector.ConnectorSplit; import io.trino.spi.predicate.TupleDomain; +import java.util.Arrays; import java.util.List; +import java.util.Objects; import java.util.Optional; import java.util.OptionalLong; @@ -49,7 +51,8 @@ public record IcebergSplit( @JsonProperty("fileStatisticsDomain") TupleDomain fileStatisticsDomain, @JsonProperty("affinityKey") Optional affinityKey, @JsonProperty("dataSequenceNumber") long dataSequenceNumber, - @JsonProperty("fileFirstRowId") OptionalLong fileFirstRowId) + @JsonProperty("fileFirstRowId") OptionalLong fileFirstRowId, + @JsonProperty("parquetFileDecryptionData") Optional parquetFileDecryptionData) implements ConnectorSplit { private static final int INSTANCE_SIZE = instanceSize(IcebergSplit.class); @@ -64,6 +67,7 @@ public record IcebergSplit( requireNonNull(fileStatisticsDomain, "fileStatisticsDomain is null"); requireNonNull(affinityKey, "affinityKey is null"); requireNonNull(fileFirstRowId, "fileFirstRowId is null"); + requireNonNull(parquetFileDecryptionData, "parquetFileDecryptionData is null"); } @Override @@ -91,7 +95,8 @@ public long getRetainedSizeInBytes() + fileStatisticsDomain.getRetainedSizeInBytes(IcebergColumnHandle::getRetainedSizeInBytes) + SIZE_OF_LONG // dataSequenceNumber + sizeOf(affinityKey, SizeOf::estimatedSizeOf) - + (fileFirstRowId.isPresent() ? SIZE_OF_LONG : 0); + + (fileFirstRowId.isPresent() ? SIZE_OF_LONG : 0) + + sizeOf(parquetFileDecryptionData, ParquetFileDecryptionData::getRetainedSizeInBytes); } @Override @@ -109,4 +114,40 @@ 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"); + } + + @Override + public boolean equals(Object o) + { + return o instanceof ParquetFileDecryptionData other + && Arrays.equals(fileEncryptionKey, other.fileEncryptionKey) + && Arrays.equals(fileAadPrefix, other.fileAadPrefix); + } + + @Override + public int hashCode() + { + return Objects.hash(Arrays.hashCode(fileEncryptionKey), Arrays.hashCode(fileAadPrefix)); + } + + @Override + public String toString() + { + return "[REDACTED]"; + } + + 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/IcebergSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/IcebergSplitSource.java index 08fa3a7c2ece..854a6e2030da 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 @@ -33,9 +33,11 @@ import io.trino.plugin.base.metrics.DurationTiming; import io.trino.plugin.base.metrics.IntList; 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; +import io.trino.spi.TrinoException; import io.trino.spi.block.Block; import io.trino.spi.connector.ColumnHandle; import io.trino.spi.connector.ConnectorSession; @@ -63,9 +65,15 @@ import org.apache.iceberg.Schema; import org.apache.iceberg.StructLike; import org.apache.iceberg.Table; +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; @@ -117,6 +125,7 @@ import static io.trino.plugin.iceberg.StructLikeWrapperWithFieldIdToIndex.createStructLikeWrapper; import static io.trino.plugin.iceberg.TypeConverter.toIcebergType; import static io.trino.plugin.iceberg.TypeConverter.toTrinoType; +import static io.trino.spi.StandardErrorCode.NOT_SUPPORTED; import static io.trino.spi.type.DateTimeEncoding.packDateTimeWithZone; import static io.trino.spi.type.TimeZoneKey.UTC_KEY; import static io.trino.spi.type.TypeUtils.writeNativeValue; @@ -126,7 +135,10 @@ import static java.util.concurrent.TimeUnit.MILLISECONDS; import static org.apache.iceberg.FileContent.EQUALITY_DELETES; import static org.apache.iceberg.FileContent.POSITION_DELETES; +import static org.apache.iceberg.FileFormat.PARQUET; +import static org.apache.iceberg.encryption.EncryptedFiles.encryptedInput; import static org.apache.iceberg.types.Conversions.fromByteBuffer; +import static org.apache.iceberg.util.ByteBuffers.toByteArray; public class IcebergSplitSource implements ConnectorSplitSource @@ -139,6 +151,8 @@ public class IcebergSplitSource private final ConnectorSession session; private final IcebergTableHandle tableHandle; private final Map fileIoProperties; + private final FileIO fileIo; + private final EncryptionManager encryptionManager; private final Scan tableScan; private final OptionalLong maxScannedFileSizeInBytes; private final Map fieldIdToType; @@ -208,7 +222,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.fileIo = requireNonNull(icebergTable.io(), "fileIo is null"); + this.fileIoProperties = requireNonNull(fileIo.properties(), "fileIoProperties is null"); + this.encryptionManager = requireNonNull(icebergTable.encryption(), "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()); @@ -587,6 +603,16 @@ private FileScanTaskWithDomain createFileScanTaskWithDomain(FileScanTask wholeFi .filter(column -> predicatedColumnIds.contains(column.fieldId())) .map(column -> getColumnHandle(column, typeManager)) .collect(toImmutableList()); + List deleteFiles = wholeFileTask.deletes().stream() + .peek(file -> verifyDeletionVectorReferencesDataFile(wholeFileTask, file)) + .map(file -> DeleteFile.fromIceberg(file, parquetFileDecryptionData( + file.format(), + file.location(), + file.fileSizeInBytes(), + file.keyMetadata(), + fileIo, + encryptionManager))) + .collect(toImmutableList()); return new FileScanTaskWithDomain( wholeFileTask, createFileStatisticsDomain( @@ -594,17 +620,73 @@ private FileScanTaskWithDomain createFileScanTaskWithDomain(FileScanTask wholeFi wholeFileTask.file().lowerBounds(), wholeFileTask.file().upperBounds(), wholeFileTask.file().nullValueCounts(), - predicatedColumns)); + predicatedColumns), + parquetFileDecryptionData( + wholeFileTask.file().format(), + wholeFileTask.file().location(), + wholeFileTask.file().fileSizeInBytes(), + wholeFileTask.file().keyMetadata(), + fileIo, + encryptionManager), + deleteFiles); } - private record FileScanTaskWithDomain(FileScanTask fileScanTask, TupleDomain fileStatisticsDomain) + private record FileScanTaskWithDomain( + FileScanTask fileScanTask, + TupleDomain fileStatisticsDomain, + Optional parquetFileDecryptionData, + List deleteFiles) { Iterator split(long targetSplitSize) { return Iterators.transform( fileScanTask().split(targetSplitSize).iterator(), - task -> new FileScanTaskWithDomain(task, fileStatisticsDomain)); + task -> new FileScanTaskWithDomain(task, fileStatisticsDomain, parquetFileDecryptionData, deleteFiles)); + } + } + + @VisibleForTesting + public static Optional parquetFileDecryptionData( + FileFormat fileFormat, + String location, + long fileSizeInBytes, + @Nullable ByteBuffer keyMetadata, + FileIO fileIo, + EncryptionManager encryptionManager) + { + requireNonNull(fileFormat, "fileFormat is null"); + requireNonNull(location, "location is null"); + requireNonNull(fileIo, "fileIo is null"); + requireNonNull(encryptionManager, "encryptionManager is null"); + + if (keyMetadata == null) { + return Optional.empty(); + } + + if (fileFormat != PARQUET) { + throw new TrinoException(NOT_SUPPORTED, "Reading encrypted non-Parquet file is not supported: " + location); } + + InputFile encryptedInputFile = fileIo.newInputFile(location, fileSizeInBytes); + return parquetFileDecryptionData( + encryptedInput(encryptedInputFile, keyMetadata.duplicate()), + encryptionManager); + } + + private static Optional parquetFileDecryptionData( + EncryptedInputFile encryptedInputFile, + EncryptionManager encryptionManager) + { + InputFile inputFile = encryptionManager.decrypt(encryptedInputFile); + + if (!(inputFile instanceof NativeEncryptionInputFile nativeEncryptionInputFile)) { + return Optional.empty(); + } + + NativeEncryptionKeyMetadata nativeKeyMetadata = nativeEncryptionInputFile.keyMetadata(); + ByteBuffer encryptionKey = requireNonNull(nativeKeyMetadata.encryptionKey(), "native encryption key is null"); + ByteBuffer aadPrefix = requireNonNull(nativeKeyMetadata.aadPrefix(), "native AAD prefix is null"); + return Optional.of(new ParquetFileDecryptionData(toByteArray(encryptionKey), toByteArray(aadPrefix))); } @VisibleForTesting @@ -738,15 +820,13 @@ private IcebergSplit toIcebergSplit(FileScanTaskWithDomain taskWithDomain) IcebergFileFormat.fromIceberg(task.file().format()), task.spec().specId(), getPartitionBlockValues(task, typeManager), - task.deletes().stream() - .peek(file -> verifyDeletionVectorReferencesDataFile(task, file)) - .map(DeleteFile::fromIceberg) - .collect(toImmutableList()), + taskWithDomain.deleteFiles(), SplitWeight.fromProportion(clamp(getSplitWeight(task), minimumAssignedSplitWeight, 1.0)), taskWithDomain.fileStatisticsDomain(), affinityKey, task.file().dataSequenceNumber(), - task.file().firstRowId() == null ? OptionalLong.empty() : OptionalLong.of(task.file().firstRowId())); + task.file().firstRowId() == null ? OptionalLong.empty() : OptionalLong.of(task.file().firstRowId()), + taskWithDomain.parquetFileDecryptionData()); } private static List getPartitionBlockValues(FileScanTask task, TypeManager typeManager) 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..0862643357cc 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 @@ -50,6 +50,7 @@ import static java.lang.String.format; import static java.util.Locale.ENGLISH; import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; import static org.apache.iceberg.TableProperties.FORMAT_VERSION; import static org.apache.iceberg.TableProperties.ORC_BLOOM_FILTER_COLUMNS; import static org.apache.iceberg.TableProperties.ORC_BLOOM_FILTER_FPP; @@ -98,6 +99,7 @@ public class IcebergTableProperties .add(ORC_BLOOM_FILTER_FPP) .add(DEFAULT_FILE_FORMAT) .add(FORMAT_VERSION) + .add(ENCRYPTION_TABLE_KEY) .build(); private final List> tableProperties; 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..6bce4ef88a1f 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,8 @@ import io.trino.metastore.HiveType; import io.trino.metastore.StorageFormat; import io.trino.plugin.iceberg.IcebergExceptions; +import io.trino.plugin.iceberg.encryption.EncryptionAwareFileIO; +import io.trino.plugin.iceberg.encryption.EncryptionManagerFactory; import io.trino.plugin.iceberg.util.HiveSchemaUtil; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; @@ -29,6 +31,8 @@ import jakarta.annotation.Nullable; import org.apache.iceberg.TableMetadata; import org.apache.iceberg.TableMetadataParser; +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; @@ -85,11 +89,15 @@ public abstract class AbstractIcebergTableOperations protected final Optional owner; protected final Optional location; private final FileIO fileIo; + private final EncryptionManagerFactory encryptionManagerFactory; protected TableMetadata currentMetadata; protected String currentMetadataLocation; protected boolean shouldRefresh = true; protected OptionalInt version = OptionalInt.empty(); + private Map encryptionProperties = Map.of(); + private EncryptionManager encryptionManager = PlaintextEncryptionManager.instance(); + private FileIO effectiveFileIo; protected AbstractIcebergTableOperations( FileIO fileIo, @@ -97,7 +105,8 @@ protected AbstractIcebergTableOperations( String database, String table, Optional owner, - Optional location) + Optional location, + EncryptionManagerFactory encryptionManagerFactory) { this.fileIo = requireNonNull(fileIo, "fileIo is null"); this.session = requireNonNull(session, "session is null"); @@ -105,6 +114,8 @@ protected AbstractIcebergTableOperations( this.tableName = requireNonNull(table, "table is null"); this.owner = requireNonNull(owner, "owner is null"); this.location = requireNonNull(location, "location is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); + this.effectiveFileIo = fileIo; } @Override @@ -115,6 +126,7 @@ public void initializeFromMetadata(TableMetadata tableMetadata) currentMetadataLocation = tableMetadata.metadataFileLocation(); shouldRefresh = false; version = OptionalInt.of(parseVersion(Location.of(currentMetadataLocation).fileName())); + updateEncryptionManager(tableMetadata); } @Override @@ -192,7 +204,13 @@ public void commit(@Nullable TableMetadata base, TableMetadata metadata) @Override public FileIO io() { - return fileIo; + return effectiveFileIo; + } + + @Override + public EncryptionManager encryption() + { + return encryptionManager; } @Override @@ -284,6 +302,7 @@ protected void refreshFromMetadataLocation(String newLocation, Function toHiveColumns(List columns) Map.of())) .collect(toImmutableList()); } + + private void updateEncryptionManager(TableMetadata metadata) + { + Map newEncryptionProperties = metadata.properties(); + if (encryptionProperties.equals(newEncryptionProperties)) { + return; + } + encryptionProperties = newEncryptionProperties; + encryptionManager = encryptionManagerFactory.create(newEncryptionProperties); + if (encryptionManager instanceof PlaintextEncryptionManager) { + effectiveFileIo = fileIo; + } + else { + effectiveFileIo = new EncryptionAwareFileIO(fileIo, encryptionManager); + } + } } 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..803ab1b2cbb3 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.EncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.TableMetadata; @@ -48,9 +49,10 @@ public FileMetastoreTableOperations( String database, String table, Optional owner, - Optional location) + Optional location, + EncryptionManagerFactory encryptionManagerFactory) { - super(fileIo, metastore, session, database, table, owner, location); + super(fileIo, metastore, session, database, table, owner, location, encryptionManagerFactory); } @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..480833114809 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.EncryptionManagerFactory; 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 EncryptionManagerFactory encryptionManagerFactory; @Inject public FileMetastoreTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, - ForwardingFileIoFactory fileIoFactory) + ForwardingFileIoFactory fileIoFactory, + EncryptionManagerFactory encryptionManagerFactory) { this.fileSystemFactory = requireNonNull(fileSystemFactory, "fileSystemFactory is null"); this.fileIoFactory = requireNonNull(fileIoFactory, "fileIoFactory is null"); + this.encryptionManagerFactory = requireNonNull(encryptionManagerFactory, "encryptionManagerFactory is null"); } @Override @@ -58,6 +62,7 @@ public IcebergTableOperations createTableOperations( database, table, owner, - location); + location, + encryptionManagerFactory); } } 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..3ac5633fef69 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.EncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.SchemaTableName; @@ -78,9 +79,10 @@ protected GlueIcebergTableOperations( String database, String table, Optional owner, - Optional location) + Optional location, + EncryptionManagerFactory encryptionManagerFactory) { - super(fileIo, session, database, table, owner, location); + super(fileIo, session, database, table, owner, location, encryptionManagerFactory); 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..407247b0a49b 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.EncryptionManagerFactory; 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 EncryptionManagerFactory encryptionManagerFactory; @Inject public GlueIcebergTableOperationsProvider( @@ -45,13 +47,15 @@ public GlueIcebergTableOperationsProvider( TypeManager typeManager, IcebergGlueCatalogConfig catalogConfig, GlueMetastoreStats stats, - GlueClient glueClient) + GlueClient glueClient, + EncryptionManagerFactory 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 @@ -75,6 +79,7 @@ public IcebergTableOperations createTableOperations( database, table, owner, - location); + location, + encryptionManagerFactory); } } 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..a15b38e9200a 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.EncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.TableNotFoundException; @@ -62,9 +63,10 @@ protected AbstractMetastoreTableOperations( String database, String table, Optional owner, - Optional location) + Optional location, + EncryptionManagerFactory encryptionManagerFactory) { - super(fileIo, session, database, table, owner, location); + super(fileIo, session, database, table, owner, location, encryptionManagerFactory); 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..2e41a25a7aae 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.EncryptionManagerFactory; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.TableNotFoundException; import org.apache.iceberg.TableMetadata; @@ -63,9 +64,10 @@ public HiveMetastoreTableOperations( String database, String table, Optional owner, - Optional location) + Optional location, + EncryptionManagerFactory encryptionManagerFactory) { - super(fileIo, metastore, session, database, table, owner, location); + super(fileIo, metastore, session, database, table, owner, location, encryptionManagerFactory); 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..5d4b1712c57c 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.EncryptionManagerFactory; 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 EncryptionManagerFactory encryptionManagerFactory; @Inject public HiveMetastoreTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, ThriftMetastoreFactory thriftMetastoreFactory, - IcebergHiveCatalogConfig metastoreConfig) + IcebergHiveCatalogConfig metastoreConfig, + EncryptionManagerFactory 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 @@ -66,6 +70,7 @@ public IcebergTableOperations createTableOperations( database, table, owner, - location); + location, + encryptionManagerFactory); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java index fe1f74d2b638..0d1bda7dce9b 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/catalog/hms/TrinoHiveCatalog.java @@ -890,7 +890,7 @@ public Metrics getMetrics() } @Override - protected void invalidateTableCache(SchemaTableName schemaTableName) + public void invalidateTableCache(SchemaTableName schemaTableName) { tableMetadataCache.invalidate(schemaTableName); } 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..2594113a4792 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.EncryptionManagerFactory; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.TableNotFoundException; import org.apache.iceberg.TableMetadata; @@ -37,9 +38,10 @@ public IcebergJdbcTableOperations( String database, String table, Optional owner, - Optional location) + Optional location, + EncryptionManagerFactory encryptionManagerFactory) { - super(fileIo, session, database, table, owner, location); + super(fileIo, session, database, table, owner, location, encryptionManagerFactory); 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..a764ab0ede39 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.EncryptionManagerFactory; 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 EncryptionManagerFactory encryptionManagerFactory; @Inject public IcebergJdbcTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, - IcebergJdbcClient jdbcClient) + IcebergJdbcClient jdbcClient, + EncryptionManagerFactory 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 @@ -60,6 +64,7 @@ public IcebergTableOperations createTableOperations( database, table, owner, - location); + location, + encryptionManagerFactory); } } 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..3fe6759c4e78 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.EncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import io.trino.spi.connector.SchemaTableName; @@ -52,9 +53,10 @@ protected IcebergNessieTableOperations( String database, String table, Optional owner, - Optional location) + Optional location, + EncryptionManagerFactory encryptionManagerFactory) { - super(fileIo, session, database, table, owner, location); + super(fileIo, session, database, table, owner, location, encryptionManagerFactory); 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..8a035557461b 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.EncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.nessie.NessieIcebergClient; @@ -33,13 +34,15 @@ public class IcebergNessieTableOperationsProvider private final TrinoFileSystemFactory fileSystemFactory; private final ForwardingFileIoFactory fileIoFactory; private final NessieIcebergClient nessieClient; + private final EncryptionManagerFactory encryptionManagerFactory; @Inject - public IcebergNessieTableOperationsProvider(TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, NessieIcebergClient nessieClient) + public IcebergNessieTableOperationsProvider(TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, NessieIcebergClient nessieClient, EncryptionManagerFactory 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 @@ -58,6 +61,7 @@ public IcebergTableOperations createTableOperations( database, table, owner, - location); + location, + encryptionManagerFactory); } } 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..587d5ebd5fda 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.EncryptionManagerFactory; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.snowflake.SnowflakeIcebergTableOperations; @@ -33,16 +34,19 @@ public class SnowflakeIcebergTableOperationsProvider private final TrinoFileSystemFactory fileSystemFactory; private final ForwardingFileIoFactory fileIoFactory; private final String snowflakeDatabase; + private final EncryptionManagerFactory encryptionManagerFactory; @Inject public SnowflakeIcebergTableOperationsProvider( TrinoFileSystemFactory fileSystemFactory, ForwardingFileIoFactory fileIoFactory, - IcebergSnowflakeCatalogConfig icebergSnowflakeCatalogConfig) + IcebergSnowflakeCatalogConfig icebergSnowflakeCatalogConfig, + EncryptionManagerFactory encryptionManagerFactory) { 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 @@ -62,6 +66,7 @@ public IcebergTableOperations createTableOperations( database, table, owner, - location); + location, + encryptionManagerFactory); } } 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..bbe9a319cb5a 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,6 +15,7 @@ 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; @@ -28,6 +29,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; @@ -43,11 +45,17 @@ public record DeleteFile( OptionalLong rowPositionUpperBound, long dataSequenceNumber, OptionalLong contentOffset, - Optional contentSizeInBytes) + Optional contentSizeInBytes, + Optional parquetFileDecryptionData) { private static final long INSTANCE_SIZE = instanceSize(DeleteFile.class); public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) + { + return fromIceberg(deleteFile, Optional.empty()); + } + + 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()); @@ -72,7 +80,8 @@ public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) rowPositionUpperBound, deleteFile.dataSequenceNumber(), contentOffset, - contentSizeInBytes); + contentSizeInBytes, + parquetFileDecryptionData); } public DeleteFile @@ -85,6 +94,7 @@ public static DeleteFile fromIceberg(org.apache.iceberg.DeleteFile deleteFile) requireNonNull(rowPositionUpperBound, "rowPositionUpperBound is null"); requireNonNull(contentOffset, "contentOffset is null"); requireNonNull(contentSizeInBytes, "contentSizeInBytes is null"); + requireNonNull(parquetFileDecryptionData, "parquetFileDecryptionData is null"); } public boolean isDeletionVector() @@ -99,7 +109,8 @@ public long retainedSizeInBytes() { return INSTANCE_SIZE + estimatedSizeOf(path) - + estimatedSizeOf(equalityFieldIds, _ -> SIZE_OF_INT); + + estimatedSizeOf(equalityFieldIds, _ -> SIZE_OF_INT) + + sizeOf(parquetFileDecryptionData, ParquetFileDecryptionData::getRetainedSizeInBytes); } @Override diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/DefaultEncryptionManagerFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/DefaultEncryptionManagerFactory.java new file mode 100644 index 000000000000..b9297cb9da25 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/DefaultEncryptionManagerFactory.java @@ -0,0 +1,68 @@ +/* + * 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.spi.TrinoException; +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.Map; +import java.util.Optional; + +import static io.trino.plugin.iceberg.IcebergErrorCode.ICEBERG_CATALOG_ERROR; +import static java.util.Objects.requireNonNull; +import static org.apache.iceberg.CatalogProperties.ENCRYPTION_KMS_IMPL; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; + +public class DefaultEncryptionManagerFactory + implements EncryptionManagerFactory +{ + private final Optional keyManagementClient; + + @Inject + public DefaultEncryptionManagerFactory(IcebergEncryptionConfig encryptionConfig) + { + requireNonNull(encryptionConfig, "encryptionConfig is null"); + this.keyManagementClient = encryptionConfig.getKmsType() + .map(kmsType -> createKeyManagementClient(kmsType.getKmsClientClassName())); + } + + public DefaultEncryptionManagerFactory(Optional keyManagementClient) + { + this.keyManagementClient = requireNonNull(keyManagementClient, "keyManagementClient is null"); + } + + @Override + public EncryptionManager create(Map tableProperties) + { + String tableKeyId = tableProperties.get(ENCRYPTION_TABLE_KEY); + if (tableKeyId == null) { + return PlaintextEncryptionManager.instance(); + } + + KeyManagementClient kmsClient = keyManagementClient.orElseThrow(() -> new TrinoException( + ICEBERG_CATALOG_ERROR, + "Can't create encryption manager, because key management client is not configured. Set iceberg.encryption.kms-type catalog property.")); + return EncryptionUtil.createEncryptionManager(tableProperties, kmsClient); + } + + private static KeyManagementClient createKeyManagementClient(String kmsClientClassName) + { + return EncryptionUtil.createKmsClient(ImmutableMap.of(ENCRYPTION_KMS_IMPL, kmsClientClassName)); + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/EncryptionAwareFileIO.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/EncryptionAwareFileIO.java new file mode 100644 index 000000000000..adf8d5f9bdbe --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/EncryptionAwareFileIO.java @@ -0,0 +1,105 @@ +/* + * 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.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.ManifestFile; +import org.apache.iceberg.encryption.EncryptingFileIO; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; + +import java.util.Map; + +/** + * Wraps {@link EncryptingFileIO} to handle encrypted manifests and data files, + * while delegating {@link #properties()} to the underlying {@link FileIO} + * since {@link EncryptingFileIO} does not support it. + *

+ * TODO: Remove after Iceberg 1.11 bump (see https://github.com/apache/iceberg/commit/473d46a) + */ +public class EncryptionAwareFileIO + implements FileIO +{ + private final FileIO delegate; + private final Map properties; + + public EncryptionAwareFileIO(FileIO fileIo, EncryptionManager encryptionManager) + { + this.delegate = EncryptingFileIO.combine(fileIo, encryptionManager); + this.properties = fileIo.properties(); + } + + @Override + public InputFile newInputFile(String path) + { + return delegate.newInputFile(path); + } + + @Override + public InputFile newInputFile(String path, long length) + { + return delegate.newInputFile(path, length); + } + + @Override + public InputFile newInputFile(DataFile file) + { + return delegate.newInputFile(file); + } + + @Override + public InputFile newInputFile(DeleteFile file) + { + return delegate.newInputFile(file); + } + + @Override + public InputFile newInputFile(ManifestFile manifest) + { + return delegate.newInputFile(manifest); + } + + @Override + public OutputFile newOutputFile(String path) + { + return delegate.newOutputFile(path); + } + + @Override + public void deleteFile(String path) + { + delegate.deleteFile(path); + } + + @Override + public Map properties() + { + return properties; + } + + @Override + public void initialize(Map newProperties) + { + delegate.initialize(newProperties); + } + + @Override + public void close() + { + delegate.close(); + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/EncryptionManagerFactory.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/EncryptionManagerFactory.java new file mode 100644 index 000000000000..4ae1dc9e172a --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/EncryptionManagerFactory.java @@ -0,0 +1,28 @@ +/* + * 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.encryption.EncryptionManager; + +import java.util.Map; + +/** + * Factory for creating Iceberg {@link EncryptionManager} instances based on table properties. + * Returns {@link org.apache.iceberg.encryption.PlaintextEncryptionManager} when the table + * is not encrypted, or an encryption-capable manager when encryption is configured. + */ +public interface EncryptionManagerFactory +{ + EncryptionManager create(Map tableProperties); +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/IcebergEncryptionConfig.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/IcebergEncryptionConfig.java new file mode 100644 index 000000000000..30dd99b0a846 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/IcebergEncryptionConfig.java @@ -0,0 +1,53 @@ +/* + * 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 io.airlift.configuration.Config; +import io.airlift.configuration.ConfigDescription; +import jakarta.validation.constraints.NotNull; + +import java.util.Optional; + +public class IcebergEncryptionConfig +{ + private Optional kmsType = Optional.empty(); + private boolean plaintextFilesAllowedForEncryptedTables; + + @NotNull + public Optional getKmsType() + { + return kmsType; + } + + @Config("iceberg.encryption.kms-type") + @ConfigDescription("Key Management Service type for Iceberg table encryption") + public IcebergEncryptionConfig setKmsType(KmsType kmsType) + { + this.kmsType = Optional.ofNullable(kmsType); + return this; + } + + public boolean isPlaintextFilesAllowedForEncryptedTables() + { + return plaintextFilesAllowedForEncryptedTables; + } + + @Config("iceberg.encryption.plaintext-files-allowed-for-encrypted-tables") + @ConfigDescription("Allow reading unencrypted files in tables with encryption enabled") + public IcebergEncryptionConfig setPlaintextFilesAllowedForEncryptedTables(boolean plaintextFilesAllowedForEncryptedTables) + { + this.plaintextFilesAllowedForEncryptedTables = plaintextFilesAllowedForEncryptedTables; + return this; + } +} diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/KmsType.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/KmsType.java new file mode 100644 index 000000000000..5a6333b96306 --- /dev/null +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/encryption/KmsType.java @@ -0,0 +1,33 @@ +/* + * 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; + +public enum KmsType +{ + AWS("org.apache.iceberg.aws.AwsKeyManagementClient"), + GCP("org.apache.iceberg.gcp.GcpKeyManagementClient"), + /**/; + + private final String kmsClientClassName; + + KmsType(String kmsClientClassName) + { + this.kmsClientClassName = kmsClientClassName; + } + + public String getKmsClientClassName() + { + return kmsClientClassName; + } +} 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 53d5d8c0f96e..33f63f7547e4 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 @@ -137,7 +137,8 @@ else if (column.getId() == DATA_CHANGE_ORDINAL_ID) { getFileIoProperties(tableCredentials), 0, OptionalLong.empty(), - functionHandle.nameMappingJson().map(NameMappingParser::fromJson)); + functionHandle.nameMappingJson().map(NameMappingParser::fromJson), + split.parquetFileDecryptionData()); 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..8b8ea443f226 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; @@ -35,7 +38,9 @@ public record TableChangesSplit( IcebergFileFormat fileFormat, String partitionSpecJson, String partitionDataJson, - SplitWeight splitWeight) implements ConnectorSplit + SplitWeight splitWeight, + Optional parquetFileDecryptionData) + implements ConnectorSplit { private static final int INSTANCE_SIZE = SizeOf.instanceSize(TableChangesSplit.class); @@ -47,6 +52,7 @@ public record TableChangesSplit( requireNonNull(partitionSpecJson, "partitionSpecJson is null"); requireNonNull(partitionDataJson, "partitionDataJson is null"); requireNonNull(splitWeight, "splitWeight is null"); + requireNonNull(parquetFileDecryptionData, "parquetFileDecryptionData is null"); } @Override @@ -62,7 +68,8 @@ public long getRetainedSizeInBytes() + estimatedSizeOf(path) + estimatedSizeOf(partitionSpecJson) + estimatedSizeOf(partitionDataJson) - + splitWeight.getRetainedSizeInBytes(); + + splitWeight.getRetainedSizeInBytes() + + SizeOf.sizeOf(parquetFileDecryptionData, ParquetFileDecryptionData::getRetainedSizeInBytes); } @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..6139a9ad0163 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,23 @@ 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.EncryptionManager; import org.apache.iceberg.io.CloseableIterable; import org.apache.iceberg.io.CloseableIterator; +import org.apache.iceberg.io.FileIO; 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; @@ -51,6 +57,8 @@ public class TableChangesSplitSource private final Table icebergTable; private final IncrementalChangelogScan tableScan; private final long targetSplitSize; + private final FileIO fileIo; + private final EncryptionManager encryptionManager; private final Closer closer = Closer.create(); private CloseableIterable changelogScanIterable; @@ -64,6 +72,8 @@ public TableChangesSplitSource( this.icebergTable = requireNonNull(icebergTable, "table is null"); this.tableScan = requireNonNull(tableScan, "tableScan is null"); this.targetSplitSize = tableScan.targetSplitSize(); + this.fileIo = requireNonNull(icebergTable.io(), "fileIo is null"); + this.encryptionManager = requireNonNull(icebergTable.encryption(), "encryptionManager is null"); } @Override @@ -153,7 +163,8 @@ private TableChangesSplit toSplit(AddedRowsScanTask task) IcebergFileFormat.fromIceberg(task.file().format()), PartitionSpecParser.toJson(task.spec()), PartitionData.toJson(task.file().partition()), - SplitWeight.standard()); + SplitWeight.standard(), + parquetFileDecryptionData(task.file())); } private TableChangesSplit toSplit(DeletedDataFileScanTask task) @@ -171,6 +182,18 @@ private TableChangesSplit toSplit(DeletedDataFileScanTask task) IcebergFileFormat.fromIceberg(task.file().format()), PartitionSpecParser.toJson(task.spec()), PartitionData.toJson(task.file().partition()), - SplitWeight.standard()); + SplitWeight.standard(), + parquetFileDecryptionData(task.file())); + } + + private Optional parquetFileDecryptionData(ContentFile file) + { + return IcebergSplitSource.parquetFileDecryptionData( + file.format(), + file.location(), + file.fileSizeInBytes(), + file.keyMetadata(), + fileIo, + encryptionManager); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTablePageSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTablePageSource.java index a4f85e1a581b..539629814038 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTablePageSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTablePageSource.java @@ -15,10 +15,8 @@ import com.google.common.io.Closer; import io.airlift.slice.Slices; -import io.trino.filesystem.TrinoFileSystem; import io.trino.plugin.iceberg.IcebergUtil; import io.trino.plugin.iceberg.StructLikeWrapperWithFieldIdToIndex; -import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.plugin.iceberg.system.FilesTable; import io.trino.plugin.iceberg.system.IcebergPartitionColumn; import io.trino.spi.Page; @@ -40,6 +38,7 @@ import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.Schema; import org.apache.iceberg.SchemaParser; +import org.apache.iceberg.io.FileIO; import org.apache.iceberg.transforms.Transforms; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; @@ -118,8 +117,7 @@ public final class FilesTablePageSource public FilesTablePageSource( TypeManager typeManager, - TrinoFileSystem trinoFileSystem, - ForwardingFileIoFactory fileIoFactory, + FileIO fileIO, List requiredColumns, FilesTableSplit split) { @@ -135,7 +133,7 @@ public FilesTablePageSource( this.primitiveFields = IcebergUtil.primitiveFields(schema).stream() .sorted(Comparator.comparing(Types.NestedField::name)) .collect(toImmutableList()); - ManifestReader> manifestReader = closer.register(readerForManifest(split.manifestFile(), fileIoFactory.create(trinoFileSystem), idToPartitionSpecMapping)); + ManifestReader> manifestReader = closer.register(readerForManifest(split.manifestFile(), fileIO, idToPartitionSpecMapping)); // TODO figure out why selecting the specific column causes null to be returned for offset_splits this.contentIterator = closer.register(requireNonNull(manifestReader, "manifestReader is null").iterator()); this.pageBuilder = new PageBuilder(requiredColumns.stream().map(column -> { diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTableSplit.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTableSplit.java index e66d14e4714b..ef13319cfc15 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTableSplit.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTableSplit.java @@ -28,7 +28,8 @@ public record FilesTableSplit( String schemaJson, String metadataTableJson, Map partitionSpecsByIdJson, - Optional partitionColumnType) + Optional partitionColumnType, + Optional encryptionKeyId) implements ConnectorSplit { private static final int INSTANCE_SIZE = instanceSize(FilesTableSplit.class); diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTableSplitSource.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTableSplitSource.java index e03eb366b237..78b70097424c 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTableSplitSource.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/FilesTableSplitSource.java @@ -31,6 +31,7 @@ import static java.util.Objects.requireNonNull; import static java.util.concurrent.CompletableFuture.completedFuture; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; public final class FilesTableSplitSource implements ConnectorSplitSource @@ -73,7 +74,8 @@ public CompletableFuture getNextBatch(int maxSize) schemaJson, metadataSchemaJson, partitionSpecsByIdJson, - partitionColumnType)); + partitionColumnType, + Optional.ofNullable(icebergTable.properties().get(ENCRYPTION_TABLE_KEY)))); } } diff --git a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/TrinoManifestFile.java b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/TrinoManifestFile.java index b29378716311..8f49b8c0cb43 100644 --- a/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/TrinoManifestFile.java +++ b/plugin/trino-iceberg/src/main/java/io/trino/plugin/iceberg/system/files/TrinoManifestFile.java @@ -16,6 +16,7 @@ import org.apache.iceberg.ManifestContent; import org.apache.iceberg.ManifestFile; +import java.nio.ByteBuffer; import java.util.List; import static io.airlift.slice.SizeOf.estimatedSizeOf; @@ -36,7 +37,8 @@ public record TrinoManifestFile( Long addedRowsCount, Long existingRowsCount, Long deletedRowsCount, - Long firstRowId) + Long firstRowId, + ByteBuffer keyMetadata) implements ManifestFile { private static final long INSTANCE_SIZE = instanceSize(TrinoManifestFile.class); @@ -69,7 +71,8 @@ public long getRetainedSizeInBytes() + sizeOf(addedRowsCount) + sizeOf(existingRowsCount) + sizeOf(deletedRowsCount) - + sizeOf(firstRowId); + + sizeOf(firstRowId) + + (keyMetadata != null ? keyMetadata.remaining() : 0); } public static TrinoManifestFile from(ManifestFile manifestFile) @@ -88,6 +91,7 @@ public static TrinoManifestFile from(ManifestFile manifestFile) manifestFile.addedRowsCount(), manifestFile.existingRowsCount(), manifestFile.deletedRowsCount(), - manifestFile.firstRowId()); + manifestFile.firstRowId(), + manifestFile.keyMetadata()); } } 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..c85d58cc3a52 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.EncryptionManagerFactory; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorSession; import org.apache.iceberg.TableMetadata; @@ -42,9 +43,10 @@ public SnowflakeIcebergTableOperations( String database, String table, Optional owner, - Optional location) + Optional location, + EncryptionManagerFactory encryptionManagerFactory) { - super(fileIo, session, database, table, owner, location); + super(fileIo, session, database, table, owner, location, encryptionManagerFactory); 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/IcebergQueryRunner.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java index 9a30adbafde1..d70dd72f46fd 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/IcebergQueryRunner.java @@ -16,6 +16,7 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; import com.google.common.io.Resources; +import com.google.inject.Module; import io.airlift.http.server.testing.TestingHttpServer; import io.airlift.json.JsonMapperProvider; import io.airlift.log.Level; @@ -100,6 +101,7 @@ public static class Builder private ImmutableMap.Builder icebergProperties = ImmutableMap.builder(); private Optional schemaInitializer = Optional.of(SchemaInitializer.builder().build()); private boolean tpcdsCatalogEnabled; + private Optional additionalOverrideModule = Optional.empty(); protected Builder() { @@ -160,6 +162,12 @@ public Builder setTpcdsCatalogEnabled(boolean tpcdsCatalogEnabled) return self(); } + public Builder setAdditionalOverrideModule(Module additionalOverrideModule) + { + this.additionalOverrideModule = Optional.of(requireNonNull(additionalOverrideModule, "additionalOverrideModule is null")); + return self(); + } + @Override public DistributedQueryRunner build() throws Exception @@ -180,7 +188,7 @@ public DistributedQueryRunner build() } Path dataDir = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data"); - queryRunner.installPlugin(new TestingIcebergPlugin(dataDir)); + queryRunner.installPlugin(new TestingIcebergPlugin(dataDir, Optional::empty, () -> additionalOverrideModule)); queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", icebergProperties.buildOrThrow()); schemaInitializer.ifPresent(initializer -> initializer.accept(queryRunner)); 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..57a9d69b7736 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,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import io.trino.plugin.iceberg.fileio.ForwardingInputFile; import io.trino.spi.block.Block; @@ -89,6 +91,7 @@ public final class IcebergTestUtils public static final ConnectorSession SESSION = TestingConnectorSession.builder() .setPropertyMetadata(new IcebergSessionProperties( new IcebergConfig(), + new IcebergEncryptionConfig(), new OrcReaderConfig(), new OrcWriterConfig(), new ParquetReaderConfig(), @@ -230,7 +233,7 @@ public static BaseTable loadTable(String tableName, String catalogName, String schemaName) { - IcebergTableOperationsProvider tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY); + IcebergTableOperationsProvider tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, new DefaultEncryptionManagerFactory(Optional.of(new TestingFileMetastoreKeyManagementClient()))); TrinoCatalog catalog = getTrinoCatalog(metastore, fileSystemFactory, catalogName); return loadIcebergTable(catalog, tableOperationsProvider, SESSION, new SchemaTableName(schemaName, tableName)); } @@ -240,7 +243,7 @@ public static TrinoCatalog getTrinoCatalog( TrinoFileSystemFactory fileSystemFactory, String catalogName) { - IcebergTableOperationsProvider tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY); + IcebergTableOperationsProvider tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, new DefaultEncryptionManagerFactory(Optional.of(new TestingFileMetastoreKeyManagementClient()))); CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); return new TrinoHiveCatalog( new CatalogName(catalogName), 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..3add2eb2ae97 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,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.SchemaTableName; import io.trino.testing.AbstractTestQueryFramework; @@ -51,7 +53,7 @@ protected QueryRunner createQueryRunner() HiveMetastore metastore = getHiveMetastore(queryRunner); CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); TrinoFileSystemFactory fileSystemFactory = getFileSystemFactory(queryRunner); - tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY); + tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, new DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())); 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 023498d261f9..eba55d5fbbd3 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,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.spi.Page; import io.trino.spi.SplitWeight; import io.trino.spi.block.BlockBuilder; @@ -153,7 +155,8 @@ public void testDynamicSplitPruningOnUnpartitionedTable() TupleDomain.all(), Optional.empty(), 0, - OptionalLong.empty()); + OptionalLong.empty(), + Optional.empty()); String tablePath = inputFile.location().fileName(); TableHandle tableHandle = new TableHandle( @@ -216,7 +219,8 @@ public void testDynamicSplitPruningOnUnpartitionedTable() TupleDomain.withColumnDomains(ImmutableMap.of(keyColumnHandle, Domain.singleValue(INTEGER, (long) keyColumnValue))), Optional.empty(), 0, - OptionalLong.empty()); + OptionalLong.empty(), + Optional.empty()); tableHandle = new TableHandle( TEST_CATALOG_HANDLE, @@ -327,7 +331,8 @@ public void testDynamicSplitPruningWithExplicitPartitionFilter() TupleDomain.all(), Optional.empty(), 0, - OptionalLong.empty()); + OptionalLong.empty(), + Optional.empty()); String tablePath = inputFile.location().fileName(); TableHandle tableHandle = new TableHandle( @@ -481,7 +486,8 @@ public void testDynamicSplitPruningWithExplicitPartitionFilterPartitionEvolution TupleDomain.all(), Optional.empty(), 0, - OptionalLong.empty()); + OptionalLong.empty(), + Optional.empty()); String tablePath = inputFile.location().fileName(); // Simulate the situation where `month` column is added at a later phase as partitioning column @@ -587,7 +593,8 @@ private static ConnectorPageSource createTestingPageSource( stats, ORC_READER_CONFIG, PARQUET_READER_CONFIG, - TESTING_TYPE_MANAGER); + TESTING_TYPE_MANAGER, + new DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())); return factory.createPageSourceProvider().createPageSource( transaction, getSession(icebergConfig), @@ -601,7 +608,7 @@ private static ConnectorPageSource createTestingPageSource( private static TestingConnectorSession getSession(IcebergConfig icebergConfig) { return TestingConnectorSession.builder() - .setPropertyMetadata(new IcebergSessionProperties(icebergConfig, ORC_READER_CONFIG, ORC_WRITER_CONFIG, PARQUET_READER_CONFIG, PARQUET_WRITER_CONFIG).getSessionProperties()) + .setPropertyMetadata(new IcebergSessionProperties(icebergConfig, new IcebergEncryptionConfig(), ORC_READER_CONFIG, ORC_WRITER_CONFIG, PARQUET_READER_CONFIG, PARQUET_WRITER_CONFIG).getSessionProperties()) .build(); } 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..2710c5de5449 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,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.SchemaTableName; @@ -85,7 +87,7 @@ 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); + tableOperationsProvider = new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, new DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())); 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 fd02127d72ed..ccdf8c897379 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,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.spi.Page; import io.trino.spi.block.Block; import io.trino.spi.block.BlockBuilder; @@ -109,6 +111,7 @@ void testMemoryTrackingWithEqualityDeletes(@TempDir Path tempDir) OptionalLong.empty(), 1L, // dataSequenceNumber OptionalLong.empty(), + Optional.empty(), Optional.empty()); IcebergPageSourceProvider provider = createPageSourceProvider(); @@ -118,6 +121,7 @@ void testMemoryTrackingWithEqualityDeletes(@TempDir Path tempDir) TestingConnectorSession session = TestingConnectorSession.builder() .setPropertyMetadata(new IcebergSessionProperties( new IcebergConfig(), + new IcebergEncryptionConfig(), ORC_READER_CONFIG, new OrcWriterConfig(), PARQUET_READER_CONFIG, @@ -144,6 +148,7 @@ void testMemoryTrackingWithEqualityDeletes(@TempDir Path tempDir) ImmutableMap.of(), 0L, // dataSequenceNumber OptionalLong.empty(), + Optional.empty(), Optional.empty())) { // Memory should still be 0 before reading any pages (lazy loading) assertThat(provider.getMemoryUsage()).isEqualTo(0); @@ -191,6 +196,7 @@ private static IcebergPageSourceProvider createPageSourceProvider() new FileFormatDataSourceStats(), ORC_READER_CONFIG.toOrcReaderOptions(), PARQUET_READER_CONFIG.toParquetReaderOptions(), - TESTING_TYPE_MANAGER); + TESTING_TYPE_MANAGER, + new DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetEncryption.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetEncryption.java new file mode 100644 index 000000000000..1bbcb7c55c43 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestIcebergParquetEncryption.java @@ -0,0 +1,412 @@ +/* + * 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 com.google.common.collect.ImmutableMap; +import io.trino.Session; +import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.catalog.hms.TrinoHiveCatalog; +import io.trino.plugin.iceberg.encryption.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.EncryptionManagerFactory; +import io.trino.spi.connector.SchemaTableName; +import io.trino.testing.AbstractTestQueryFramework; +import io.trino.testing.QueryRunner; +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.SortOrder; +import org.apache.iceberg.Table; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.types.Types; +import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; +import org.junit.jupiter.api.TestInstance; + +import java.util.List; +import java.util.Optional; + +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; +import static io.trino.plugin.iceberg.IcebergTestUtils.SESSION; +import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; +import static io.trino.plugin.iceberg.IcebergTestUtils.getHiveMetastore; +import static io.trino.plugin.iceberg.IcebergTestUtils.getTrinoCatalog; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.createEncryptionManager; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.createTestRecords; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.createTestRecordsWithName; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.writeEncryptedAvroDataFile; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.writeEncryptedDataFile; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.writeEncryptedEqualityDeleteFile; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.writeEncryptedPositionDeleteFile; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.writePlaintextDataFile; +import static io.trino.plugin.iceberg.util.EncryptedFileTestUtils.writePlaintextFileWithEncryptionKeyMetadata; +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.apache.iceberg.TableProperties.DEFAULT_FILE_FORMAT; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; +import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; + +@TestInstance(PER_CLASS) +public class TestIcebergParquetEncryption + extends AbstractTestQueryFramework +{ + private static final Schema TABLE_SCHEMA = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "age", Types.IntegerType.get())); + + private static final String TEST_KEY_ID = "test-key-id"; + private static final TestingFileMetastoreKeyManagementClient KMS_CLIENT = new TestingFileMetastoreKeyManagementClient(); + + private TrinoCatalog catalog; + + @Override + protected QueryRunner createQueryRunner() + throws Exception + { + return IcebergQueryRunner.builder() + .setAdditionalOverrideModule(binder -> newOptionalBinder(binder, EncryptionManagerFactory.class) + .setBinding() + .toInstance(new DefaultEncryptionManagerFactory(Optional.of(KMS_CLIENT)))) + .build(); + } + + @BeforeAll + public void setUp() + { + catalog = getTrinoCatalog(getHiveMetastore(getQueryRunner()), getFileSystemFactory(getQueryRunner()), "iceberg"); + getQueryRunner().execute("CREATE SCHEMA IF NOT EXISTS tpch"); + } + + @Test + public void testReadEncryptedParquetTable() + throws Exception + { + String tableName = "enc_parquet_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + createIcebergTable(schemaTableName, TABLE_SCHEMA); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + + DataFile dataFile = writeEncryptedDataFile(table, encryptionManager, createTestRecords(TABLE_SCHEMA, 100)); + table.newFastAppend().appendFile(dataFile).commit(); + invalidateTableCache(schemaTableName); + + assertThat(computeActual("SELECT bool_and(key_metadata IS NOT NULL) FROM \"" + tableName + "$files\"").getOnlyValue()) + .isEqualTo(true); + assertThat(computeActual("SELECT count(*), min(id), max(id), min(age), max(age) FROM " + tableName) + .getMaterializedRows().get(0).getFields()) + .containsExactly(100L, 0, 99, 0, 99); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testReadEncryptedParquetTableWithPositionDeletes() + throws Exception + { + String tableName = "enc_parquet_pos_delete_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + createIcebergTable(schemaTableName, TABLE_SCHEMA); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + + DataFile dataFile = writeEncryptedDataFile(table, encryptionManager, createTestRecords(TABLE_SCHEMA, 100)); + table.newFastAppend().appendFile(dataFile).commit(); + + // Delete rows at positions 0 and 1 (id=0, id=1) + DeleteFile posDeleteFile = writeEncryptedPositionDeleteFile( + table, encryptionManager, dataFile.path().toString(), 0, 1); + table.newRowDelta().addDeletes(posDeleteFile).commit(); + invalidateTableCache(schemaTableName); + + assertThat(computeActual("SELECT count(*) FROM " + tableName).getOnlyValue()) + .isEqualTo(98L); + assertThat(computeActual("SELECT min(id) FROM " + tableName).getOnlyValue()) + .isEqualTo(2); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testReadEncryptedParquetTableWithEqualityDeletes() + throws Exception + { + String tableName = "enc_parquet_eq_delete_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + createIcebergTable(schemaTableName, TABLE_SCHEMA); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + + DataFile dataFile = writeEncryptedDataFile(table, encryptionManager, createTestRecords(TABLE_SCHEMA, 100)); + table.newFastAppend().appendFile(dataFile).commit(); + + // Equality delete all rows where age=99 (that's id=0) + Schema deleteRowSchema = TABLE_SCHEMA.select("age"); + List equalityFieldIds = ImmutableList.of(TABLE_SCHEMA.findField("age").fieldId()); + Record deleteRecord = GenericRecord.create(deleteRowSchema); + deleteRecord.setField("age", 99); + + DeleteFile eqDeleteFile = writeEncryptedEqualityDeleteFile( + table, encryptionManager, deleteRowSchema, equalityFieldIds, ImmutableList.of(deleteRecord)); + table.newRowDelta().addDeletes(eqDeleteFile).commit(); + invalidateTableCache(schemaTableName); + + assertThat(computeActual("SELECT count(*) FROM " + tableName).getOnlyValue()) + .isEqualTo(99L); + assertThat(computeActual("SELECT count(*) FROM " + tableName + " WHERE age = 99").getOnlyValue()) + .isEqualTo(0L); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testPlaintextFileWithKeyMetadataFails() + throws Exception + { + Schema schemaWithString = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get())); + + String tableName = "enc_plaintext_fail_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + createIcebergTable(schemaTableName, schemaWithString); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + + DataFile fakeEncryptedFile = writePlaintextFileWithEncryptionKeyMetadata( + table, encryptionManager, createTestRecordsWithName(schemaWithString, 100)); + table.newFastAppend().appendFile(fakeEncryptedFile).commit(); + invalidateTableCache(schemaTableName); + + assertThatThrownBy(() -> computeActual("SELECT count(*) FROM " + tableName + " WHERE name = 'name_1'")) + .hasMessageContaining("Applying decryptor on plaintext file"); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testPlaintextFileWithKeyMetadataAllowedBySessionProperty() + throws Exception + { + Schema schemaWithString = new Schema( + Types.NestedField.required(1, "id", Types.IntegerType.get()), + Types.NestedField.required(2, "name", Types.StringType.get())); + + String tableName = "enc_plaintext_allowed_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + createIcebergTable(schemaTableName, schemaWithString); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + + DataFile fakeEncryptedFile = writePlaintextFileWithEncryptionKeyMetadata( + table, encryptionManager, createTestRecordsWithName(schemaWithString, 100)); + table.newFastAppend().appendFile(fakeEncryptedFile).commit(); + invalidateTableCache(schemaTableName); + + Session allowPlaintext = Session.builder(getSession()) + .setCatalogSessionProperty("iceberg", "plaintext_files_allowed_for_encrypted_tables", "true") + .build(); + assertThat(computeActual(allowPlaintext, "SELECT count(*) FROM " + tableName + " WHERE name = 'name_1'").getOnlyValue()) + .isEqualTo(1L); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testReadMixedEncryptedAndPlaintextFiles() + throws Exception + { + String tableName = "enc_mixed_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + createIcebergTable(schemaTableName, TABLE_SCHEMA); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + + // Write a plaintext file before encryption is enabled + DataFile plaintextFile = writePlaintextDataFile(table, createTestRecords(TABLE_SCHEMA, 50)); + table.newFastAppend().appendFile(plaintextFile).commit(); + + // Enable encryption and write an encrypted file + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + DataFile encryptedFile = writeEncryptedDataFile(table, encryptionManager, createTestRecords(TABLE_SCHEMA, 50)); + table.newFastAppend().appendFile(encryptedFile).commit(); + invalidateTableCache(schemaTableName); + + // Both files should be readable — plaintext file has no key_metadata so no decryption is attempted + assertThat(computeActual("SELECT count(*) FROM " + tableName).getOnlyValue()) + .isEqualTo(100L); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testReadEncryptedAvroTableFails() + throws Exception + { + String tableName = "enc_avro_fail_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + catalog.newCreateTableTransaction( + SESSION, + schemaTableName, + TABLE_SCHEMA, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + Optional.of(catalog.defaultTableLocation(SESSION, schemaTableName)), + ImmutableMap.of(DEFAULT_FILE_FORMAT, "AVRO")) + .commitTransaction(); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + + DataFile dataFile = writeEncryptedAvroDataFile(table, encryptionManager, createTestRecords(TABLE_SCHEMA, 10)); + table.newFastAppend().appendFile(dataFile).commit(); + invalidateTableCache(schemaTableName); + + assertThatThrownBy(() -> computeActual("SELECT * FROM " + tableName)) + .hasMessageContaining("Reading encrypted non-Parquet file is not supported"); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testWriteToEncryptedTableFails() + throws Exception + { + String tableName = "enc_write_fail_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + createIcebergTable(schemaTableName, TABLE_SCHEMA); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + + // Write data via Iceberg API so DELETE has rows to process + DataFile dataFile = writeEncryptedDataFile(table, encryptionManager, createTestRecords(TABLE_SCHEMA, 10)); + table.newFastAppend().appendFile(dataFile).commit(); + invalidateTableCache(schemaTableName); + + assertThatThrownBy(() -> getQueryRunner().execute("INSERT INTO " + tableName + " VALUES (1, 2)")) + .hasMessageContaining("Writing to encrypted Iceberg tables is not supported"); + assertThatThrownBy(() -> getQueryRunner().execute("DELETE FROM " + tableName + " WHERE id = 1")) + .hasMessageContaining("Writing to encrypted Iceberg tables is not supported"); + assertThatThrownBy(() -> getQueryRunner().execute("MERGE INTO " + tableName + " USING (VALUES 1) t(id) ON " + tableName + ".id = t.id WHEN MATCHED THEN DELETE")) + .hasMessageContaining("Writing to encrypted Iceberg tables is not supported"); + assertThatThrownBy(() -> getQueryRunner().execute("ALTER TABLE " + tableName + " EXECUTE optimize")) + .hasMessageContaining("Writing to encrypted Iceberg tables is not supported"); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + @Test + public void testReadEncryptedTableChanges() + throws Exception + { + String tableName = "enc_table_changes_" + randomNameSuffix(); + SchemaTableName schemaTableName = new SchemaTableName("tpch", tableName); + + createIcebergTable(schemaTableName, TABLE_SCHEMA); + try { + Table table = catalog.loadTable(SESSION, schemaTableName); + setTableEncryptionKey(table, schemaTableName); + EncryptionManager encryptionManager = createEncryptionManager(TEST_KEY_ID, KMS_CLIENT); + + // Write first snapshot with encrypted data + DataFile dataFile1 = writeEncryptedDataFile(table, encryptionManager, createTestRecords(TABLE_SCHEMA, 50)); + table.newFastAppend().appendFile(dataFile1).commit(); + invalidateTableCache(schemaTableName); + long startSnapshotId = table.currentSnapshot().snapshotId(); + + // Write second snapshot with more encrypted data + DataFile dataFile2 = writeEncryptedDataFile(table, encryptionManager, createTestRecords(TABLE_SCHEMA, 30)); + table.newFastAppend().appendFile(dataFile2).commit(); + invalidateTableCache(schemaTableName); + long endSnapshotId = table.currentSnapshot().snapshotId(); + + // Read table changes between snapshots + assertThat(computeActual( + "SELECT count(*) FROM TABLE(system.table_changes('tpch', '%s', %d, %d))".formatted(tableName, startSnapshotId, endSnapshotId)) + .getOnlyValue()) + .isEqualTo(30L); + } + finally { + getQueryRunner().execute("DROP TABLE IF EXISTS " + tableName); + } + } + + private void createIcebergTable(SchemaTableName schemaTableName, Schema schema) + { + catalog.newCreateTableTransaction( + SESSION, + schemaTableName, + schema, + PartitionSpec.unpartitioned(), + SortOrder.unsorted(), + Optional.of(catalog.defaultTableLocation(SESSION, schemaTableName)), + ImmutableMap.of(DEFAULT_FILE_FORMAT, "PARQUET")) + .commitTransaction(); + } + + private void setTableEncryptionKey(Table table, SchemaTableName schemaTableName) + { + table.updateProperties() + .set(ENCRYPTION_TABLE_KEY, TEST_KEY_ID) + .commit(); + invalidateTableCache(schemaTableName); + } + + private void invalidateTableCache(SchemaTableName schemaTableName) + { + ((TrinoHiveCatalog) catalog).invalidateTableCache(schemaTableName); + } +} 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 354fc55f2dee..11c35ef58546 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 @@ -26,9 +26,12 @@ 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.IcebergSplit.ParquetFileDecryptionData; 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.spi.SplitWeight; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ColumnHandle; @@ -43,6 +46,7 @@ import io.trino.testing.AbstractTestQueryFramework; import io.trino.testing.QueryRunner; import io.trino.testing.TestingConnectorSession; +import org.apache.iceberg.FileFormat; import org.apache.iceberg.PartitionSpec; import org.apache.iceberg.PartitionSpecParser; import org.apache.iceberg.SchemaParser; @@ -53,12 +57,24 @@ import org.apache.iceberg.data.parquet.GenericParquetWriter; import org.apache.iceberg.deletes.PositionDelete; import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedInputFile; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionKeyMetadata; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.EncryptionTestHelpers; +import org.apache.iceberg.encryption.NativeEncryptionInputFile; +import org.apache.iceberg.encryption.NativeEncryptionKeyMetadata; +import org.apache.iceberg.encryption.NativeEncryptionOutputFile; import org.apache.iceberg.io.FileIO; +import org.apache.iceberg.io.InputFile; +import org.apache.iceberg.io.OutputFile; +import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.metrics.InMemoryMetricsReporter; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.types.Conversions; import org.apache.iceberg.types.Type; import org.apache.iceberg.types.Types; +import org.apache.iceberg.util.ByteBuffers; import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.Timeout; @@ -79,6 +95,7 @@ import static com.google.common.util.concurrent.MoreExecutors.newDirectExecutorService; import static io.trino.metastore.cache.CachingHiveMetastore.createPerTransactionCache; import static io.trino.plugin.iceberg.IcebergSplitSource.createFileStatisticsDomain; +import static io.trino.plugin.iceberg.IcebergSplitSource.parquetFileDecryptionData; import static io.trino.plugin.iceberg.IcebergTestUtils.FILE_IO_FACTORY; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; import static io.trino.plugin.iceberg.IcebergTestUtils.getHiveMetastore; @@ -88,7 +105,9 @@ import static io.trino.tpch.TpchTable.NATION; import static io.trino.type.InternalTypeManager.TESTING_TYPE_MANAGER; import static java.util.concurrent.TimeUnit.SECONDS; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; import static org.assertj.core.api.Assertions.assertThat; +import static org.assertj.core.api.Assertions.assertThatThrownBy; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; @TestInstance(PER_CLASS) @@ -98,6 +117,7 @@ public class TestIcebergSplitSource private static final ConnectorSession SESSION = TestingConnectorSession.builder() .setPropertyMetadata(new IcebergSessionProperties( new IcebergConfig(), + new IcebergEncryptionConfig(), new OrcReaderConfig(), new OrcWriterConfig(), new ParquetReaderConfig(), @@ -107,6 +127,7 @@ public class TestIcebergSplitSource private TrinoFileSystemFactory fileSystemFactory; private TrinoCatalog catalog; + private HiveMetastore metastore; @Override protected QueryRunner createQueryRunner() @@ -116,7 +137,7 @@ protected QueryRunner createQueryRunner() .setInitialTables(NATION) .build(); - HiveMetastore metastore = getHiveMetastore(queryRunner); + metastore = getHiveMetastore(queryRunner); this.fileSystemFactory = getFileSystemFactory(queryRunner); CachingHiveMetastore cachingHiveMetastore = createPerTransactionCache(metastore, 1000); @@ -127,7 +148,7 @@ protected QueryRunner createQueryRunner() fileSystemFactory, FILE_IO_FACTORY, TESTING_TYPE_MANAGER, - new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY), + new FileMetastoreTableOperationsProvider(fileSystemFactory, FILE_IO_FACTORY, new DefaultEncryptionManagerFactory(Optional.of(new TestingFileMetastoreKeyManagementClient()))), false, false, false, @@ -224,6 +245,7 @@ public void testFileStatisticsDomain() IcebergSplit split = generateSplit(nationTable, tableHandle, DynamicFilter.EMPTY); assertThat(split.fileStatisticsDomain()).isEqualTo(TupleDomain.all()); + assertThat(split.parquetFileDecryptionData()).isEmpty(); IcebergColumnHandle nationKey = IcebergColumnHandle.optional(new ColumnIdentity(1, "nationkey", ColumnIdentity.TypeCategory.PRIMITIVE, ImmutableList.of())) .columnType(BIGINT) @@ -402,6 +424,216 @@ public void testSplitWeight() assertThat(split.getSplitWeight().getRawValue()).isGreaterThan(splitWeightWithPositionDelete.getRawValue()); } + @Test + public void testParquetFileDecryptionData() + { + byte[] expectedKey = new byte[] {1, 2, 3}; + byte[] expectedAadPrefix = new byte[] {4, 5, 6}; + ByteBuffer keyMetadata = ByteBuffer.wrap(new byte[] {7, 8, 9}); + + FileIO fileIo = new FileIO() + { + @Override + public InputFile newInputFile(String path) + { + return new TestingInputFile(path); + } + + @Override + public InputFile newInputFile(String path, long length) + { + return new TestingInputFile(path); + } + + @Override + public OutputFile newOutputFile(String path) + { + throw new UnsupportedOperationException(); + } + + @Override + public void deleteFile(String path) + { + throw new UnsupportedOperationException(); + } + }; + + EncryptionManager encryptionManager = new EncryptionManager() + { + @Override + public InputFile decrypt(EncryptedInputFile encrypted) + { + return new NativeEncryptionInputFile() + { + @Override + public InputFile encryptedInputFile() + { + return encrypted.encryptedInputFile(); + } + + @Override + public NativeEncryptionKeyMetadata keyMetadata() + { + return new NativeEncryptionKeyMetadata() + { + @Override + public ByteBuffer encryptionKey() + { + return ByteBuffer.wrap(expectedKey); + } + + @Override + public ByteBuffer aadPrefix() + { + return ByteBuffer.wrap(expectedAadPrefix); + } + + @Override + public ByteBuffer buffer() + { + return keyMetadata; + } + + @Override + public EncryptionKeyMetadata copy() + { + return this; + } + }; + } + + @Override + public long getLength() + { + return encrypted.encryptedInputFile().getLength(); + } + + @Override + public SeekableInputStream newStream() + { + throw new UnsupportedOperationException(); + } + + @Override + public String location() + { + return encrypted.encryptedInputFile().location(); + } + + @Override + public boolean exists() + { + return encrypted.encryptedInputFile().exists(); + } + }; + } + + @Override + public EncryptedOutputFile encrypt(OutputFile rawOutput) + { + throw new UnsupportedOperationException(); + } + }; + + assertThatThrownBy(() -> parquetFileDecryptionData( + FileFormat.ORC, + "memory:///test.orc", + 10, + keyMetadata, + fileIo, + encryptionManager)) + .hasMessageContaining("Reading encrypted non-Parquet file is not supported"); + + assertThat(parquetFileDecryptionData( + FileFormat.PARQUET, + "memory:///test.parquet", + 10, + null, + fileIo, + encryptionManager)).isEmpty(); + + Optional decryptionData = parquetFileDecryptionData( + FileFormat.PARQUET, + "memory:///test.parquet", + 10, + keyMetadata, + fileIo, + encryptionManager); + + assertThat(decryptionData).isPresent(); + assertThat(decryptionData.orElseThrow().fileEncryptionKey()).containsExactly(expectedKey); + assertThat(decryptionData.orElseThrow().fileAadPrefix()).containsExactly(expectedAadPrefix); + } + + @Test + public void testParquetFileDecryptionDataWithStandardKeyMetadataManager() + { + FileIO fileIo = FILE_IO_FACTORY.create(fileSystemFactory.create(SESSION)); + String location = "local:///standard-key-metadata-manager-" + UUID.randomUUID() + ".parquet"; + SchemaTableName nationTable = new SchemaTableName("tpch", "nation"); + Table icebergTable = catalog.loadTable(SESSION, nationTable); + icebergTable.updateProperties().set(ENCRYPTION_TABLE_KEY, "test-key-id").commit(); + ((TrinoHiveCatalog) catalog).invalidateTableCache(nationTable); + try { + EncryptionManager standardEncryptionManager = EncryptionTestHelpers.createEncryptionManager(); + NativeEncryptionOutputFile encryptedOutputFile = (NativeEncryptionOutputFile) standardEncryptionManager.encrypt(fileIo.newOutputFile(location)); + byte[] expectedKey = ByteBuffers.toByteArray(encryptedOutputFile.keyMetadata().encryptionKey()); + byte[] expectedAadPrefix = ByteBuffers.toByteArray(encryptedOutputFile.keyMetadata().aadPrefix()); + EncryptionManager tableEncryptionManager = catalog.loadTable(SESSION, nationTable).encryption(); + + Optional decryptionData = parquetFileDecryptionData( + FileFormat.PARQUET, + location, + 10, + encryptedOutputFile.keyMetadata().buffer(), + fileIo, + tableEncryptionManager); + + assertThat(decryptionData).isPresent(); + assertThat(decryptionData.orElseThrow().fileEncryptionKey()).containsExactly(expectedKey); + assertThat(decryptionData.orElseThrow().fileAadPrefix()).containsExactly(expectedAadPrefix); + } + finally { + icebergTable.updateProperties().remove(ENCRYPTION_TABLE_KEY).commit(); + ((TrinoHiveCatalog) catalog).invalidateTableCache(nationTable); + } + } + + private static class TestingInputFile + implements InputFile + { + private final String path; + + private TestingInputFile(String path) + { + this.path = path; + } + + @Override + public long getLength() + { + return 0; + } + + @Override + public SeekableInputStream newStream() + { + throw new UnsupportedOperationException(); + } + + @Override + public String location() + { + return path; + } + + @Override + public boolean exists() + { + return true; + } + } + private IcebergSplit generateSplit(Table nationTable, IcebergTableHandle tableHandle, DynamicFilter dynamicFilter) throws Exception { 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 8a591a88d3e6..3738c471b63c 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 @@ -23,6 +23,8 @@ import io.trino.metastore.HiveMetastore; import io.trino.plugin.hive.HivePlugin; import io.trino.plugin.iceberg.catalog.TrinoCatalog; +import io.trino.plugin.iceberg.encryption.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.EncryptionManagerFactory; import io.trino.plugin.tpch.TpchPlugin; import io.trino.spi.connector.SchemaTableName; import io.trino.testing.AbstractTestQueryFramework; @@ -67,6 +69,7 @@ import static com.google.common.collect.Iterables.getOnlyElement; import static com.google.common.io.MoreFiles.deleteRecursively; import static com.google.common.io.RecursiveDeleteOption.ALLOW_INSECURE; +import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; import static io.trino.plugin.iceberg.IcebergQueryRunner.ICEBERG_CATALOG; import static io.trino.plugin.iceberg.IcebergTestUtils.SESSION; import static io.trino.plugin.iceberg.IcebergTestUtils.getFileSystemFactory; @@ -110,7 +113,12 @@ protected QueryRunner createQueryRunner() dataDirectory = queryRunner.getCoordinator().getBaseDataDir().resolve("iceberg_data"); dataDirectory.toFile().mkdirs(); - queryRunner.installPlugin(new TestingIcebergPlugin(dataDirectory)); + queryRunner.installPlugin(new TestingIcebergPlugin( + dataDirectory, + Optional::empty, + () -> Optional.of(binder -> newOptionalBinder(binder, EncryptionManagerFactory.class) + .setBinding() + .toInstance(new DefaultEncryptionManagerFactory(Optional.of(new TestingFileMetastoreKeyManagementClient())))))); queryRunner.createCatalog(ICEBERG_CATALOG, "iceberg", ImmutableMap.of( "iceberg.catalog.type", "TESTING_FILE_METASTORE", "iceberg.format-version", "3", @@ -1240,7 +1248,7 @@ private BaseTable loadTable(String tableName) } @Test - void testV3RejectsEncryptionKeyProperty() + void testV3AllowsEncryptionKeyPropertyForReads() { String tableName = "test_v3_encryption_" + randomNameSuffix(); assertUpdate("CREATE TABLE " + tableName + " (id INTEGER) WITH (format = 'ORC', format_version = 3)"); @@ -1252,24 +1260,14 @@ void testV3RejectsEncryptionKeyProperty() .set("encryption.key-id", "test_key") .commit(); - assertQueryFails( - "SELECT * FROM " + tableName, - ".*Iceberg table encryption is not supported.*"); - - // Also verify INSERT fails with encryption key set - assertQueryFails( - "INSERT INTO " + tableName + " VALUES 2", - ".*Iceberg table encryption is not supported.*"); + assertThat(query("SELECT * FROM " + tableName)) + .matches("VALUES 1"); - // Clean up by removing the property first - icebergTable.updateProperties() - .remove("encryption.key-id") - .commit(); assertUpdate("DROP TABLE " + tableName); } @Test - void testV3RejectsEncryptionKeysInMetadata() + void testV3AllowsEncryptionKeysInMetadataForReads() throws Exception { String temp = "tmp_v3_encryption_src_" + randomNameSuffix(); @@ -1301,14 +1299,14 @@ void testV3RejectsEncryptionKeysInMetadata() assertUpdate("CALL system.register_table(CURRENT_SCHEMA, '%s', '%s')" .formatted(registered, hadoopTableLocation)); - assertQueryFails( - "SELECT * FROM " + registered, - ".*Iceberg table encryption is not supported.*"); + assertThat(query("SELECT * FROM " + registered)) + .matches("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)); + assertUpdate("DROP TABLE " + registered); assertUpdate("DROP TABLE " + temp); - deleteRecursively(hadoopTableLocation, ALLOW_INSECURE); + if (Files.exists(hadoopTableLocation)) { + deleteRecursively(hadoopTableLocation, ALLOW_INSECURE); + } } @Test diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestParquetPredicates.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestParquetPredicates.java index 9a403fdc3cd0..da4a0f149b6b 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestParquetPredicates.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestParquetPredicates.java @@ -15,10 +15,13 @@ import com.google.common.collect.ImmutableList; import com.google.common.collect.ImmutableMap; +import io.trino.parquet.crypto.FileDecryptionProperties; +import io.trino.plugin.iceberg.IcebergSplit.ParquetFileDecryptionData; import io.trino.spi.predicate.Domain; import io.trino.spi.predicate.TupleDomain; import io.trino.spi.type.RowType; import org.apache.parquet.column.ColumnDescriptor; +import org.apache.parquet.hadoop.metadata.ColumnPath; import org.apache.parquet.schema.GroupType; import org.apache.parquet.schema.MessageType; import org.apache.parquet.schema.PrimitiveType; @@ -26,6 +29,7 @@ import java.util.List; import java.util.Map; +import java.util.Optional; import static io.trino.parquet.ParquetTypeUtils.getDescriptors; import static io.trino.plugin.iceberg.ColumnIdentity.TypeCategory.PRIMITIVE; @@ -204,4 +208,23 @@ public void testParquetTupleDomainStructWithMissingPrimitiveColumn() assertThat(calculatedTupleDomain.isAll()).isTrue(); } + + @Test + public void testParquetFileDecryptionProperties() + { + byte[] fileKey = new byte[] {1, 2, 3}; + byte[] aadPrefix = new byte[] {4, 5, 6}; + + assertThat(IcebergPageSourceProvider.createParquetFileDecryptionProperties(Optional.empty(), false)).isEmpty(); + + Optional fileDecryptionProperties = IcebergPageSourceProvider.createParquetFileDecryptionProperties( + Optional.of(new ParquetFileDecryptionData(fileKey, aadPrefix)), false); + assertThat(fileDecryptionProperties).isPresent(); + assertThat(fileDecryptionProperties.orElseThrow().getAadPrefix()).isPresent(); + assertThat(fileDecryptionProperties.orElseThrow().getAadPrefix().orElseThrow()).containsExactly(aadPrefix); + assertThat(fileDecryptionProperties.orElseThrow().getKeyRetriever().getFooterKey(Optional.empty())) + .hasValueSatisfying(key -> assertThat(key).containsExactly(fileKey)); + assertThat(fileDecryptionProperties.orElseThrow().getKeyRetriever().getColumnKey(ColumnPath.fromDotString("c"), Optional.empty())) + .hasValueSatisfying(key -> assertThat(key).containsExactly(fileKey)); + } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingFileMetastoreKeyManagementClient.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingFileMetastoreKeyManagementClient.java new file mode 100644 index 000000000000..30da7baf801e --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingFileMetastoreKeyManagementClient.java @@ -0,0 +1,41 @@ +/* + * 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 org.apache.iceberg.encryption.KeyManagementClient; + +import java.nio.ByteBuffer; +import java.util.Map; + +public class TestingFileMetastoreKeyManagementClient + implements KeyManagementClient +{ + @Override + public ByteBuffer wrapKey(ByteBuffer key, String wrappingKeyId) + { + return key.duplicate(); + } + + @Override + public ByteBuffer unwrapKey(ByteBuffer wrappedKey, String wrappingKeyId) + { + return wrappedKey.duplicate(); + } + + @Override + public void initialize(Map properties) + { + // no-op + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergConnectorFactory.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergConnectorFactory.java index 36bf7010a978..d305fe07f8b5 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergConnectorFactory.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergConnectorFactory.java @@ -15,6 +15,7 @@ import com.google.common.collect.ImmutableMap; import com.google.inject.Module; +import io.airlift.configuration.ConfigurationAwareModule; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.local.LocalFileSystemFactory; import io.trino.plugin.hive.metastore.file.FileHiveMetastoreConfig; @@ -37,20 +38,30 @@ public class TestingIcebergConnectorFactory { private final Path localFileSystemRootPath; private final Supplier> icebergCatalogModule; + private final Supplier> additionalOverrideModule; public TestingIcebergConnectorFactory(Path localFileSystemRootPath) { - this(localFileSystemRootPath, Optional::empty); + this(localFileSystemRootPath, Optional::empty, Optional::empty); } @Deprecated public TestingIcebergConnectorFactory( Path localFileSystemRootPath, Supplier> icebergCatalogModule) + { + this(localFileSystemRootPath, icebergCatalogModule, Optional::empty); + } + + public TestingIcebergConnectorFactory( + Path localFileSystemRootPath, + Supplier> icebergCatalogModule, + Supplier> additionalOverrideModule) { this.localFileSystemRootPath = requireNonNull(localFileSystemRootPath, "localFileSystemRootPath is null"); boolean ignored = localFileSystemRootPath.toFile().mkdirs(); this.icebergCatalogModule = requireNonNull(icebergCatalogModule, "icebergCatalogModule is null"); + this.additionalOverrideModule = requireNonNull(additionalOverrideModule, "additionalOverrideModule is null"); } @Override @@ -68,13 +79,16 @@ public Connector create(String catalogName, Map config, Connecto .put("iceberg.catalog.type", "TESTING_FILE_METASTORE") .buildOrThrow(); } - Module module = binder -> { + Module localModule = binder -> { newMapBinder(binder, String.class, TrinoFileSystemFactory.class) .addBinding("local").toInstance(new LocalFileSystemFactory(localFileSystemRootPath)); configBinder(binder).bindConfigDefaults( FileHiveMetastoreConfig.class, metastoreConfig -> metastoreConfig.setCatalogDirectory("local:///" + catalogName)); }; + Module module = additionalOverrideModule.get() + .map(override -> (Module) ConfigurationAwareModule.combine(localModule, override)) + .orElse(localModule); return createConnector(catalogName, config, context, module, icebergCatalogModule.get()); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergPlugin.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergPlugin.java index 2bd3f1c107ec..f6764aa0b7cf 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergPlugin.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/TestingIcebergPlugin.java @@ -30,17 +30,27 @@ public class TestingIcebergPlugin { private final Path localFileSystemRootPath; private final Supplier> icebergCatalogModule; + private final Supplier> additionalOverrideModule; public TestingIcebergPlugin(Path localFileSystemRootPath) { - this(localFileSystemRootPath, Optional::empty); + this(localFileSystemRootPath, Optional::empty, Optional::empty); } @Deprecated public TestingIcebergPlugin(Path localFileSystemRootPath, Supplier> icebergCatalogModule) + { + this(localFileSystemRootPath, icebergCatalogModule, Optional::empty); + } + + public TestingIcebergPlugin( + Path localFileSystemRootPath, + Supplier> icebergCatalogModule, + Supplier> additionalOverrideModule) { this.localFileSystemRootPath = requireNonNull(localFileSystemRootPath, "localFileSystemRootPath is null"); this.icebergCatalogModule = requireNonNull(icebergCatalogModule, "icebergCatalogModule is null"); + this.additionalOverrideModule = requireNonNull(additionalOverrideModule, "additionalOverrideModule is null"); } @Override @@ -49,6 +59,6 @@ public Iterable getConnectorFactories() List connectorFactories = ImmutableList.copyOf(super.getConnectorFactories()); verify(connectorFactories.size() == 1, "Unexpected connector factories: %s", connectorFactories); - return ImmutableList.of(new TestingIcebergConnectorFactory(localFileSystemRootPath, icebergCatalogModule)); + return ImmutableList.of(new TestingIcebergConnectorFactory(localFileSystemRootPath, icebergCatalogModule, additionalOverrideModule)); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java index 37152a009bcf..991889c6634e 100644 --- a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/BaseTrinoCatalogTest.java @@ -29,6 +29,7 @@ import io.trino.plugin.iceberg.IcebergMetadata; import io.trino.plugin.iceberg.IcebergSessionProperties; import io.trino.plugin.iceberg.TableStatisticsWriter; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.spi.NodeVersion; import io.trino.spi.TrinoException; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; @@ -85,6 +86,7 @@ public abstract class BaseTrinoCatalogTest protected static final ConnectorSession SESSION = TestingConnectorSession.builder() .setPropertyMetadata(new IcebergSessionProperties( new IcebergConfig(), + new IcebergEncryptionConfig(), new OrcReaderConfig(), new OrcWriterConfig(), new ParquetReaderConfig(), 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..b786282c6dce 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.encryption.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.plugin.iceberg.fileio.ForwardingFileIo; import org.apache.iceberg.io.InputFile; import org.junit.jupiter.api.Test; @@ -61,7 +63,8 @@ public InputFile newInputFile(String path) "test-database", "test-table", Optional.of("test-owner"), - Optional.empty()) + Optional.empty(), + new DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())) { // Without this, we'd have to create a table that's never accessed anyway, because we're simulating S3 errors. @Override 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..83cb315a68d3 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,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; import io.trino.spi.connector.SchemaTableName; @@ -113,7 +115,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 DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())), 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..f6e4a7810325 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,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.spi.NodeVersion; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMaterializedViewDefinition; @@ -105,7 +107,8 @@ private TrinoCatalog createGlueTrinoCatalog(boolean useUniqueTableLocations, boo TESTING_TYPE_MANAGER, catalogConfig, new GlueMetastoreStats(), - glueClient), + glueClient, + new DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())), "test", new StatsRecordingGlueClient(glueClient, new GlueMetastoreStats()), useSystemSecurity, @@ -261,7 +264,8 @@ public void testDefaultLocation() TESTING_TYPE_MANAGER, catalogConfig, new GlueMetastoreStats(), - glueClient), + glueClient, + new DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())), "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..92020f84c092 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 @@ -39,6 +39,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; 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 DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())), 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..b5ea24fa1f17 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 @@ -22,6 +22,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; 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 DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())), 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 DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())), nessieClient, icebergNessieCatalogConfig.getDefaultWarehouseDir(), false); diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestIcebergEncryptionConfig.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestIcebergEncryptionConfig.java new file mode 100644 index 000000000000..041735a5c0a6 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestIcebergEncryptionConfig.java @@ -0,0 +1,49 @@ +/* + * 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 org.junit.jupiter.api.Test; + +import java.util.Map; + +import static io.airlift.configuration.testing.ConfigAssertions.assertFullMapping; +import static io.airlift.configuration.testing.ConfigAssertions.assertRecordedDefaults; +import static io.airlift.configuration.testing.ConfigAssertions.recordDefaults; + +public class TestIcebergEncryptionConfig +{ + @Test + public void testDefaults() + { + assertRecordedDefaults(recordDefaults(IcebergEncryptionConfig.class) + .setKmsType(null) + .setPlaintextFilesAllowedForEncryptedTables(false)); + } + + @Test + public void testExplicitPropertyMappings() + { + Map properties = ImmutableMap.builder() + .put("iceberg.encryption.kms-type", "AWS") + .put("iceberg.encryption.plaintext-files-allowed-for-encrypted-tables", "true") + .buildOrThrow(); + + IcebergEncryptionConfig expected = new IcebergEncryptionConfig() + .setKmsType(KmsType.AWS) + .setPlaintextFilesAllowedForEncryptedTables(true); + + assertFullMapping(properties, expected); + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestKmsClientInstantiation.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestKmsClientInstantiation.java new file mode 100644 index 000000000000..bcaeb55d1ae1 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/encryption/TestKmsClientInstantiation.java @@ -0,0 +1,33 @@ +/* + * 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.encryption.KeyManagementClient; +import org.junit.jupiter.params.ParameterizedTest; +import org.junit.jupiter.params.provider.ValueSource; + +import static org.assertj.core.api.Assertions.assertThat; + +class TestKmsClientInstantiation +{ + @ParameterizedTest + @ValueSource(strings = {"org.apache.iceberg.aws.AwsKeyManagementClient", "org.apache.iceberg.gcp.GcpKeyManagementClient"}) + void testKmsClientClassCanBeLoaded(String kmsImpl) + throws Exception + { + Class kmsClass = Class.forName(kmsImpl); + assertThat(KeyManagementClient.class).isAssignableFrom(kmsClass); + assertThat(kmsClass.getDeclaredConstructor().newInstance()).isInstanceOf(KeyManagementClient.class); + } +} diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/util/EncryptedFileTestUtils.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/util/EncryptedFileTestUtils.java new file mode 100644 index 000000000000..793fa8550ceb --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/util/EncryptedFileTestUtils.java @@ -0,0 +1,273 @@ +/* + * 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.util; + +import org.apache.iceberg.DataFile; +import org.apache.iceberg.DataFiles; +import org.apache.iceberg.DeleteFile; +import org.apache.iceberg.Metrics; +import org.apache.iceberg.PartitionSpec; +import org.apache.iceberg.Schema; +import org.apache.iceberg.Table; +import org.apache.iceberg.avro.Avro; +import org.apache.iceberg.data.GenericRecord; +import org.apache.iceberg.data.Record; +import org.apache.iceberg.data.parquet.GenericParquetWriter; +import org.apache.iceberg.deletes.EqualityDeleteWriter; +import org.apache.iceberg.deletes.PositionDelete; +import org.apache.iceberg.deletes.PositionDeleteWriter; +import org.apache.iceberg.encryption.EncryptedOutputFile; +import org.apache.iceberg.encryption.EncryptionManager; +import org.apache.iceberg.encryption.EncryptionUtil; +import org.apache.iceberg.encryption.KeyManagementClient; +import org.apache.iceberg.io.DataWriter; +import org.apache.iceberg.parquet.Parquet; + +import java.io.Closeable; +import java.nio.file.Path; +import java.util.List; +import java.util.Map; +import java.util.stream.IntStream; + +import static io.trino.testing.TestingNames.randomNameSuffix; +import static org.apache.iceberg.TableProperties.ENCRYPTION_TABLE_KEY; + +public final class EncryptedFileTestUtils +{ + private EncryptedFileTestUtils() {} + + /** + * Creates an EncryptionManager for a table using the given KeyManagementClient. + * The table properties must contain ENCRYPTION_TABLE_KEY. + */ + public static EncryptionManager createEncryptionManager(Map tableProperties, KeyManagementClient kmsClient) + { + return EncryptionUtil.createEncryptionManager(tableProperties, kmsClient); + } + + /** + * Creates an EncryptionManager using explicit properties (not from table). + * Use when encryption key is set out-of-band (e.g. in metastore params) rather than as table property. + */ + public static EncryptionManager createEncryptionManager(String keyId, KeyManagementClient kmsClient) + { + return EncryptionUtil.createEncryptionManager( + Map.of(ENCRYPTION_TABLE_KEY, keyId), + kmsClient); + } + + public static DataFile writeEncryptedDataFile(Table table, EncryptionManager encryptionManager, List records) + throws Exception + { + String dataPath = Path.of(table.location()) + .resolve("data") + .resolve("enc-data-" + randomNameSuffix() + ".parquet") + .toString(); + + EncryptedOutputFile encryptedOutputFile = encryptionManager.encrypt(table.io().newOutputFile(dataPath)); + try (DataWriter writer = Parquet.writeData(encryptedOutputFile) + .forTable(table) + .withSpec(table.spec()) + .withPartition(null) + .withKeyMetadata(encryptedOutputFile.keyMetadata()) + .createWriterFunc(GenericParquetWriter::create) + .build()) { + for (Record record : records) { + writer.write(record); + } + writer.close(); + return writer.toDataFile(); + } + } + + public static DataFile writeEncryptedAvroDataFile(Table table, EncryptionManager encryptionManager, List records) + throws Exception + { + String dataPath = Path.of(table.location()) + .resolve("data") + .resolve("enc-data-" + randomNameSuffix() + ".avro") + .toString(); + + EncryptedOutputFile encryptedOutputFile = encryptionManager.encrypt(table.io().newOutputFile(dataPath)); + try (org.apache.iceberg.io.DataWriter writer = Avro.writeData(encryptedOutputFile) + .forTable(table) + .withSpec(table.spec()) + .withPartition(null) + .withKeyMetadata(encryptedOutputFile.keyMetadata()) + .createWriterFunc(org.apache.iceberg.data.avro.DataWriter::create) + .build()) { + for (Record record : records) { + writer.write(record); + } + writer.close(); + return writer.toDataFile(); + } + } + + public static DeleteFile writeEncryptedPositionDeleteFile( + Table table, + EncryptionManager encryptionManager, + String dataFilePath, + long... positions) + throws Exception + { + String deletePath = Path.of(table.location()) + .resolve("data") + .resolve("enc-pos-delete-" + randomNameSuffix() + ".parquet") + .toString(); + + EncryptedOutputFile encryptedOutputFile = encryptionManager.encrypt(table.io().newOutputFile(deletePath)); + try (PositionDeleteWriter writer = Parquet.writeDeletes(encryptedOutputFile) + .withSpec(PartitionSpec.unpartitioned()) + .overwrite() + .withKeyMetadata(encryptedOutputFile.keyMetadata()) + .buildPositionWriter()) { + PositionDelete positionDelete = PositionDelete.create(); + for (long position : positions) { + writer.write(positionDelete.set(dataFilePath, position, null)); + } + writer.close(); + return writer.toDeleteFile(); + } + } + + public static DeleteFile writeEncryptedEqualityDeleteFile( + Table table, + EncryptionManager encryptionManager, + Schema deleteRowSchema, + List equalityFieldIds, + List deleteRecords) + throws Exception + { + String deletePath = Path.of(table.location()) + .resolve("data") + .resolve("enc-eq-delete-" + randomNameSuffix() + ".parquet") + .toString(); + + EncryptedOutputFile encryptedOutputFile = encryptionManager.encrypt(table.io().newOutputFile(deletePath)); + EqualityDeleteWriter writer = Parquet.writeDeletes(encryptedOutputFile) + .forTable(table) + .rowSchema(deleteRowSchema) + .withSpec(PartitionSpec.unpartitioned()) + .equalityFieldIds(equalityFieldIds) + .overwrite() + .withKeyMetadata(encryptedOutputFile.keyMetadata()) + .createWriterFunc(GenericParquetWriter::create) + .buildEqualityWriter(); + + try (Closeable ignored = writer) { + for (Record record : deleteRecords) { + writer.write(record); + } + } + + return writer.toDeleteFile(); + } + + /** + * Writes a plaintext (unencrypted) Parquet data file. + */ + public static DataFile writePlaintextDataFile(Table table, List records) + throws Exception + { + String dataPath = Path.of(table.location()) + .resolve("data") + .resolve("plain-" + randomNameSuffix() + ".parquet") + .toString(); + + try (DataWriter writer = Parquet.writeData(table.io().newOutputFile(dataPath)) + .forTable(table) + .withSpec(table.spec()) + .withPartition(null) + .createWriterFunc(GenericParquetWriter::create) + .build()) { + for (Record record : records) { + writer.write(record); + } + writer.close(); + return writer.toDataFile(); + } + } + + /** + * Writes a plaintext Parquet file but attaches valid encryption key metadata, + * simulating a corrupted or tampered file. + */ + public static DataFile writePlaintextFileWithEncryptionKeyMetadata( + Table table, + EncryptionManager encryptionManager, + List records) + throws Exception + { + String dataPath = Path.of(table.location()) + .resolve("data") + .resolve("fake-enc-" + randomNameSuffix() + ".parquet") + .toString(); + + EncryptedOutputFile encryptedOutputFile = encryptionManager.encrypt(table.io().newOutputFile(dataPath)); + + try (DataWriter writer = Parquet.writeData(table.io().newOutputFile(dataPath)) + .forTable(table) + .withSpec(table.spec()) + .withPartition(null) + .createWriterFunc(GenericParquetWriter::create) + .build()) { + for (Record record : records) { + writer.write(record); + } + writer.close(); + DataFile plaintextFile = writer.toDataFile(); + + return DataFiles.builder(table.spec()) + .withPath(plaintextFile.path().toString()) + .withFileSizeInBytes(plaintextFile.fileSizeInBytes()) + .withRecordCount(plaintextFile.recordCount()) + .withFormat(plaintextFile.format()) + .withMetrics(new Metrics( + plaintextFile.recordCount(), + plaintextFile.columnSizes(), + plaintextFile.valueCounts(), + plaintextFile.nullValueCounts(), + plaintextFile.nanValueCounts(), + plaintextFile.lowerBounds(), + plaintextFile.upperBounds())) + .withEncryptionKeyMetadata(encryptedOutputFile.keyMetadata()) + .build(); + } + } + + public static List createTestRecords(Schema schema, int count) + { + return IntStream.range(0, count) + .mapToObj(i -> { + Record record = GenericRecord.create(schema); + record.setField("id", i); + record.setField("age", count - 1 - i); + return record; + }) + .toList(); + } + + public static List createTestRecordsWithName(Schema schema, int count) + { + return IntStream.range(0, count) + .mapToObj(i -> { + Record record = GenericRecord.create(schema); + record.setField("id", i); + record.setField("name", "name_" + i); + return record; + }) + .toList(); + } +} 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..7da6064b1ffd 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 @@ -32,6 +32,8 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.spi.NodeVersion; import io.trino.spi.catalog.CatalogName; import io.trino.spi.connector.ConnectorMetadata; @@ -183,7 +185,7 @@ 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, CATALOG_CONFIG, new DefaultEncryptionManagerFactory(new IcebergEncryptionConfig())); 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..eaaed5533546 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,9 @@ 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.DefaultEncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.EncryptionManagerFactory; +import io.trino.plugin.iceberg.encryption.IcebergEncryptionConfig; import io.trino.plugin.iceberg.fileio.ForwardingFileIoFactory; import static com.google.inject.multibindings.OptionalBinder.newOptionalBinder; @@ -55,6 +58,8 @@ public class LakehouseIcebergModule protected void setup(Binder binder) { configBinder(binder).bindConfig(IcebergConfig.class); + configBinder(binder).bindConfig(IcebergEncryptionConfig.class); + binder.bind(EncryptionManagerFactory.class).to(DefaultEncryptionManagerFactory.class).in(Scopes.SINGLETON); binder.bind(IcebergNodePartitioningProvider.class).in(Scopes.SINGLETON); binder.bind(IcebergPageSinkProvider.class).in(Scopes.SINGLETON); diff --git a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvSinglenodeSparkIceberg.java b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvSinglenodeSparkIceberg.java index 4afe0d59cbac..ec2aee076e6d 100644 --- a/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvSinglenodeSparkIceberg.java +++ b/testing/trino-product-tests-launcher/src/main/java/io/trino/tests/product/launcher/env/environment/EnvSinglenodeSparkIceberg.java @@ -32,6 +32,7 @@ import static io.trino.tests.product.launcher.docker.ContainerUtil.forSelectedPorts; import static io.trino.tests.product.launcher.env.EnvironmentContainers.HADOOP; import static io.trino.tests.product.launcher.env.EnvironmentContainers.TESTS; +import static io.trino.tests.product.launcher.env.EnvironmentContainers.isTrinoContainer; import static io.trino.tests.product.launcher.env.EnvironmentDefaults.HADOOP_BASE_IMAGE; import static io.trino.tests.product.launcher.env.common.Hadoop.CONTAINER_HADOOP_INIT_D; import static java.util.Objects.requireNonNull; @@ -44,6 +45,16 @@ public class EnvSinglenodeSparkIceberg private static final File HIVE_JDBC_PROVIDER = new File("testing/trino-product-tests-launcher/target/hive-jdbc.jar"); private static final int SPARK_THRIFT_PORT = 10213; + private static final int LOCALSTACK_PORT = 4566; + private static final String LOCALSTACK_CONTAINER = "localstack"; + private static final String LOCALSTACK_ENDPOINT_URL = "http://" + LOCALSTACK_CONTAINER + ":" + LOCALSTACK_PORT; + private static final String AWS_REGION = "us-east-1"; + private static final String AWS_ACCESS_KEY_ID = "test"; + private static final String AWS_SECRET_ACCESS_KEY = "test"; + // Spark engages Iceberg encryption only with the 1.11.0+ catalog wiring + // (iceberg/apache#13066, #15272). Replace with 1.11.0 once released. + private static final String ICEBERG_SNAPSHOT_VERSION = "1.11.0-SNAPSHOT"; + private static final String ICEBERG_SNAPSHOT_REPO = "https://repository.apache.org/content/repositories/snapshots/"; private final DockerFiles dockerFiles; private final PortBinder portBinder; @@ -70,19 +81,52 @@ public void extendEnvironment(Environment.Builder builder) builder.addConnector("iceberg", forHostPath(dockerFiles.getDockerFilesHostPath("conf/environment/singlenode-spark-iceberg/iceberg.properties"))); + builder.configureContainers(container -> { + if (isTrinoContainer(container.getLogicalName())) { + container + .withEnv("AWS_ACCESS_KEY_ID", AWS_ACCESS_KEY_ID) + .withEnv("AWS_SECRET_ACCESS_KEY", AWS_SECRET_ACCESS_KEY) + .withEnv("AWS_REGION", AWS_REGION) + .withEnv("AWS_ENDPOINT_URL_KMS", LOCALSTACK_ENDPOINT_URL); + } + }); + + builder.addContainer(createLocalStack()); builder.addContainer(createSpark()) - .containerDependsOn("spark", HADOOP); + .containerDependsOn("spark", HADOOP) + .containerDependsOn("spark", LOCALSTACK_CONTAINER); builder.configureContainer(TESTS, dockerContainer -> dockerContainer + .withEnv("AWS_ACCESS_KEY_ID", AWS_ACCESS_KEY_ID) + .withEnv("AWS_SECRET_ACCESS_KEY", AWS_SECRET_ACCESS_KEY) + .withEnv("AWS_REGION", AWS_REGION) + .withEnv("AWS_ENDPOINT_URL_KMS", LOCALSTACK_ENDPOINT_URL) // Binding instead of copying for avoiding OutOfMemoryError https://github.com/testcontainers/testcontainers-java/issues/2863 .withFileSystemBind(HIVE_JDBC_PROVIDER.getParent(), "/docker/jdbc", BindMode.READ_ONLY)); } + @SuppressWarnings("resource") + private DockerContainer createLocalStack() + { + DockerContainer container = new DockerContainer("localstack/localstack:4.14.0", LOCALSTACK_CONTAINER) + .withEnv("SERVICES", "kms") + .withStartupCheckStrategy(new IsRunningStartupCheckStrategy()) + .waitingFor(forSelectedPorts(LOCALSTACK_PORT)); + + portBinder.exposePort(container, LOCALSTACK_PORT); + + return container; + } + @SuppressWarnings("resource") private DockerContainer createSpark() { DockerContainer container = new DockerContainer("ghcr.io/trinodb/testing/spark4-iceberg:" + hadoopImagesVersion, "spark") .withEnv("HADOOP_USER_NAME", "hive") + .withEnv("AWS_ACCESS_KEY_ID", AWS_ACCESS_KEY_ID) + .withEnv("AWS_SECRET_ACCESS_KEY", AWS_SECRET_ACCESS_KEY) + .withEnv("AWS_REGION", AWS_REGION) + .withEnv("AWS_ENDPOINT_URL_KMS", LOCALSTACK_ENDPOINT_URL) .withCopyFileToContainer( forHostPath(dockerFiles.getDockerFilesHostPath("conf/environment/singlenode-spark-iceberg/spark-defaults.conf")), "/spark/conf/spark-defaults.conf") @@ -90,13 +134,32 @@ private DockerContainer createSpark() forHostPath(dockerFiles.getDockerFilesHostPath("common/spark/log4j2.properties")), "/spark/conf/log4j2.properties") .withCommand( - "spark-submit", - "--master", "local[*]", - "--class", "org.apache.spark.sql.hive.thriftserver.HiveThriftServer2", - "--name", "Thrift JDBC/ODBC Server", - "--packages", "org.apache.spark:spark-avro_2.12:3.2.1", - "--conf", "spark.hive.server2.thrift.port=" + SPARK_THRIFT_PORT, - "spark-internal") + "bash", "-c", + // The bundled iceberg-spark-runtime in the spark4-iceberg image is too old + // to engage encryption (see ICEBERG_SNAPSHOT_VERSION). Replace it (and add + // a matching iceberg-aws-bundle) by fetching the latest SNAPSHOT directly + // into /spark/jars so they are on the driver classpath at startup. + // --packages alone is not enough: the Thrift server's catalog plugin loader + // doesn't see jars added via spark.jars at session init time. + "set -e; " + + "REPO=" + ICEBERG_SNAPSHOT_REPO + "org/apache/iceberg; " + + "fetch_latest() { " + + " local artifact=$1; " + + " local version=$(curl -sSL \"$REPO/$artifact/" + ICEBERG_SNAPSHOT_VERSION + "/maven-metadata.xml\" " + + " | grep -oE '1\\.11\\.0-[0-9.-]+' | head -1 | sed -E 's|||g'); " + + " curl -sSL --fail -o \"/spark/jars/$artifact-" + ICEBERG_SNAPSHOT_VERSION + ".jar\" " + + " \"$REPO/$artifact/" + ICEBERG_SNAPSHOT_VERSION + "/$artifact-${version}.jar\"; " + + "}; " + + "rm -f /spark/jars/iceberg-spark-runtime-*.jar /spark/jars/iceberg-aws-bundle-*.jar; " + + "fetch_latest iceberg-spark-runtime-4.0_2.13; " + + "fetch_latest iceberg-aws-bundle; " + + "exec spark-submit" + + " --master local[*]" + + " --class org.apache.spark.sql.hive.thriftserver.HiveThriftServer2" + + " --name 'Thrift JDBC/ODBC Server'" + + " --packages org.apache.spark:spark-avro_2.12:3.2.1" + + " --conf spark.hive.server2.thrift.port=" + SPARK_THRIFT_PORT + + " spark-internal") .withStartupCheckStrategy(new IsRunningStartupCheckStrategy()) .waitingFor(forSelectedPorts(SPARK_THRIFT_PORT)); diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/singlenode-spark-iceberg/iceberg.properties b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/singlenode-spark-iceberg/iceberg.properties index 357466f9c384..41030b302be4 100644 --- a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/singlenode-spark-iceberg/iceberg.properties +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/singlenode-spark-iceberg/iceberg.properties @@ -2,4 +2,5 @@ connector.name=iceberg hive.metastore.uri=thrift://hadoop-master:9083 iceberg.register-table-procedure.enabled=true iceberg.allowed-extra-properties=* +iceberg.encryption.kms-type=AWS fs.hadoop.enabled=true diff --git a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/singlenode-spark-iceberg/spark-defaults.conf b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/singlenode-spark-iceberg/spark-defaults.conf index ab609b7fa072..eef3072edf7a 100644 --- a/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/singlenode-spark-iceberg/spark-defaults.conf +++ b/testing/trino-product-tests-launcher/src/main/resources/docker/trino-product-tests/conf/environment/singlenode-spark-iceberg/spark-defaults.conf @@ -10,6 +10,7 @@ spark.sql.catalog.iceberg_test.type=hive spark.sql.catalog.iceberg_test.uri=thrift://hadoop-master:9083 ; disabling caching allows us to run spark queries interchangeably with trino's spark.sql.catalog.iceberg_test.cache-enabled=false +spark.sql.catalog.iceberg_test.encryption.kms-type=aws spark.sql.extensions=org.apache.iceberg.spark.extensions.IcebergSparkSessionExtensions spark.hadoop.fs.defaultFS=hdfs://hadoop-master:9000 diff --git a/testing/trino-product-tests/pom.xml b/testing/trino-product-tests/pom.xml index 88eac80ccf98..30c7acf567ec 100644 --- a/testing/trino-product-tests/pom.xml +++ b/testing/trino-product-tests/pom.xml @@ -218,6 +218,11 @@ glue + + software.amazon.awssdk + kms + + software.amazon.awssdk regions diff --git a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java index 821f5fff5e5a..a86b46a1e773 100644 --- a/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java +++ b/testing/trino-product-tests/src/main/java/io/trino/tests/product/iceberg/TestIcebergSparkCompatibility.java @@ -32,6 +32,10 @@ import org.testng.SkipException; import org.testng.annotations.DataProvider; import org.testng.annotations.Test; +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.math.BigDecimal; import java.net.URI; @@ -2438,6 +2442,50 @@ public void testAlterTableExecuteProceduresOnEmptyTable() assertThat(onTrino().executeQuery("SELECT * FROM " + trinoTableName)).hasNoRows(); } + @Test(groups = {ICEBERG, PROFILE_SPECIFIC_TESTS}) + public void testTrinoReadingSparkEncryptedData() + { + String baseTableName = "test_trino_reading_spark_encrypted_" + randomNameSuffix(); + String trinoTableName = trinoTableName(baseTableName); + String sparkTableName = sparkTableName(baseTableName); + + String keyId; + try (KmsClient kmsClient = createLocalStackKmsClient()) { + keyId = kmsClient.createKey(builder -> builder.description("trino-iceberg-pt-" + randomNameSuffix())).keyMetadata().keyId(); + } + + onSpark().executeQuery("DROP TABLE IF EXISTS " + sparkTableName); + + onSpark().executeQuery(format( + "CREATE TABLE %s (id INT, name STRING) USING ICEBERG " + + "TBLPROPERTIES ('write.format.default'='PARQUET', 'format-version'='3', 'encryption.key-id'='%s')", + sparkTableName, keyId)); + onSpark().executeQuery(format("INSERT INTO %s VALUES (1, 'alice'), (2, 'bob'), (3, 'charlie')", sparkTableName)); + + assertThat(onTrino().executeQuery("SELECT * FROM " + trinoTableName + " ORDER BY id")) + .containsOnly(row(1, "alice"), row(2, "bob"), row(3, "charlie")); + + assertThat(onSpark().executeQuery("SELECT * FROM " + sparkTableName + " ORDER BY id")) + .containsOnly(row(1, "alice"), row(2, "bob"), row(3, "charlie")); + + assertThat(onTrino().executeQuery("SELECT count(*) FROM " + trinoTableName("\"" + baseTableName + "$files\"") + " WHERE key_metadata IS NULL")) + .containsOnly(row(0L)); + + assertQueryFailure(() -> onTrino().executeQuery("INSERT INTO " + trinoTableName + " VALUES (4, 'dave')")) + .hasMessageContaining("Writing to encrypted Iceberg tables is not supported"); + + onSpark().executeQuery("DROP TABLE IF EXISTS " + sparkTableName); + } + + private static KmsClient createLocalStackKmsClient() + { + return KmsClient.builder() + .endpointOverride(URI.create("http://localstack:4566")) + .credentialsProvider(StaticCredentialsProvider.create(AwsBasicCredentials.create("test", "test"))) + .region(Region.US_EAST_1) + .build(); + } + private static String escapeSparkString(String value) { return value.replace("\\", "\\\\").replace("'", "\\'");