diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java index c7b5b5d41c74..7932d8400c58 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTSessionCatalog.java @@ -51,6 +51,8 @@ import org.apache.iceberg.catalog.Namespace; import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.encryption.EncryptionUtil; +import org.apache.iceberg.encryption.KeyManagementClient; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.NoSuchNamespaceException; import org.apache.iceberg.exceptions.NoSuchTableException; @@ -170,6 +172,7 @@ public class RESTSessionCatalog extends BaseViewSessionCatalog private CloseableGroup closeables = null; private Set endpoints; private Supplier> mutationHeaders = Map::of; + private KeyManagementClient keyManagementClient = null; private String namespaceSeparator = null; private RESTTableCache tableCache; @@ -275,6 +278,13 @@ public void initialize(String name, Map unresolved) { mergedProps, RESTCatalogProperties.METRICS_REPORTING_ENABLED, RESTCatalogProperties.METRICS_REPORTING_ENABLED_DEFAULT); + + if (mergedProps.containsKey(CatalogProperties.ENCRYPTION_KMS_TYPE) + || mergedProps.containsKey(CatalogProperties.ENCRYPTION_KMS_IMPL)) { + this.keyManagementClient = EncryptionUtil.createKmsClient(mergedProps); + this.closeables.addCloseable(this.keyManagementClient); + } + this.namespaceSeparator = PropertyUtil.propertyAsString( mergedProps, @@ -573,6 +583,7 @@ private Supplier createTableSupplier( Map::of, mutationHeaders, tableFileIO(context, tableConf, credentials), + keyManagementClient, tableMetadata, endpoints); @@ -716,6 +727,7 @@ public Table registerTable( Map::of, mutationHeaders, tableFileIO(context, tableConf, response.credentials()), + keyManagementClient, response.tableMetadata(), endpoints); @@ -985,6 +997,7 @@ public Table create() { Map::of, mutationHeaders, tableFileIO(context, tableConf, response.credentials()), + keyManagementClient, response.tableMetadata(), endpoints); @@ -1018,6 +1031,7 @@ public Transaction createTransaction() { Map::of, mutationHeaders, tableFileIO(context, tableConf, response.credentials()), + keyManagementClient, RESTTableOperations.UpdateType.CREATE, createChanges(meta), meta, @@ -1083,6 +1097,7 @@ public Transaction replaceTransaction() { Map::of, mutationHeaders, tableFileIO(context, tableConf, response.credentials()), + keyManagementClient, RESTTableOperations.UpdateType.REPLACE, changes.build(), base, @@ -1231,6 +1246,7 @@ private FileIO tableFileIO( * @param mutationHeaderSupplier a supplier for additional HTTP headers to include in mutation * requests (POST/DELETE) * @param fileIO the FileIO implementation for reading and writing table metadata and data files + * @param kmsClient the {@link KeyManagementClient} for encrypted tables * @param current the current table metadata * @param supportedEndpoints the set of supported REST endpoints * @return a new RESTTableOperations instance @@ -1241,10 +1257,18 @@ protected RESTTableOperations newTableOps( Supplier> readHeaders, Supplier> mutationHeaderSupplier, FileIO fileIO, + KeyManagementClient kmsClient, TableMetadata current, Set supportedEndpoints) { return new RESTTableOperations( - restClient, path, readHeaders, mutationHeaderSupplier, fileIO, current, supportedEndpoints); + restClient, + path, + readHeaders, + mutationHeaderSupplier, + fileIO, + kmsClient, + current, + supportedEndpoints); } /** @@ -1261,6 +1285,7 @@ protected RESTTableOperations newTableOps( * @param mutationHeaderSupplier a supplier for additional HTTP headers to include in mutation * requests (POST/DELETE) * @param fileIO the FileIO implementation for reading and writing table metadata and data files + * @param kmsClient the {@link KeyManagementClient} for encrypted tables * @param updateType the {@link RESTTableOperations.UpdateType} being performed * @param createChanges the list of metadata updates to apply during table creation or replacement * @param current the current table metadata (may be null for CREATE operations) @@ -1273,6 +1298,7 @@ protected RESTTableOperations newTableOps( Supplier> readHeaders, Supplier> mutationHeaderSupplier, FileIO fileIO, + KeyManagementClient kmsClient, RESTTableOperations.UpdateType updateType, List createChanges, TableMetadata current, @@ -1283,6 +1309,7 @@ protected RESTTableOperations newTableOps( readHeaders, mutationHeaderSupplier, fileIO, + kmsClient, updateType, createChanges, current, diff --git a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java index be763d30fef1..d22607d664ac 100644 --- a/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java +++ b/core/src/main/java/org/apache/iceberg/rest/RESTTableOperations.java @@ -21,9 +21,12 @@ import java.util.List; import java.util.Map; import java.util.Objects; +import java.util.Optional; import java.util.Set; import java.util.function.Consumer; import java.util.function.Supplier; +import java.util.stream.Collectors; +import org.apache.iceberg.CatalogProperties; import org.apache.iceberg.LocationProviders; import org.apache.iceberg.MetadataUpdate; import org.apache.iceberg.SnapshotRef; @@ -32,17 +35,25 @@ import org.apache.iceberg.TableProperties; import org.apache.iceberg.UpdateRequirement; import org.apache.iceberg.UpdateRequirements; +import org.apache.iceberg.encryption.EncryptedKey; +import org.apache.iceberg.encryption.EncryptingFileIO; 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 org.apache.iceberg.encryption.StandardEncryptionManager; import org.apache.iceberg.exceptions.CommitStateUnknownException; import org.apache.iceberg.io.FileIO; import org.apache.iceberg.io.LocationProvider; import org.apache.iceberg.relocated.com.google.common.base.Preconditions; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Lists; import org.apache.iceberg.rest.requests.UpdateTableRequest; import org.apache.iceberg.rest.responses.ErrorResponse; import org.apache.iceberg.rest.responses.LoadTableResponse; import org.apache.iceberg.util.LocationUtil; +import org.apache.iceberg.util.PropertyUtil; class RESTTableOperations implements TableOperations { private static final String METADATA_FOLDER_NAME = "metadata"; @@ -58,17 +69,26 @@ enum UpdateType { private final Supplier> readHeaders; private final Supplier> mutationHeaders; private final FileIO io; + private final KeyManagementClient keyManagementClient; private final List createChanges; private final TableMetadata replaceBase; private final Set endpoints; private UpdateType updateType; private TableMetadata current; + private EncryptionManager encryptionManager; + private EncryptingFileIO encryptingFileIO; + private String tableKeyId; + private int encryptionDekLength; + + private List encryptedKeys = List.of(); + RESTTableOperations( RESTClient client, String path, Supplier> headers, FileIO io, + KeyManagementClient keyManagementClient, TableMetadata current, Set endpoints) { this( @@ -77,6 +97,7 @@ enum UpdateType { headers, headers, io, + keyManagementClient, UpdateType.SIMPLE, Lists.newArrayList(), current, @@ -88,11 +109,22 @@ enum UpdateType { String path, Supplier> headers, FileIO io, + KeyManagementClient keyManagementClient, UpdateType updateType, List createChanges, TableMetadata current, Set endpoints) { - this(client, path, headers, headers, io, updateType, createChanges, current, endpoints); + this( + client, + path, + headers, + headers, + io, + keyManagementClient, + updateType, + createChanges, + current, + endpoints); } RESTTableOperations( @@ -101,6 +133,7 @@ enum UpdateType { Supplier> readHeaders, Supplier> mutationHeaders, FileIO io, + KeyManagementClient keyManagementClient, TableMetadata current, Set endpoints) { this( @@ -109,6 +142,7 @@ enum UpdateType { readHeaders, mutationHeaders, io, + keyManagementClient, UpdateType.SIMPLE, Lists.newArrayList(), current, @@ -121,6 +155,7 @@ enum UpdateType { Supplier> readHeaders, Supplier> mutationHeaders, FileIO io, + KeyManagementClient keyManagementClient, UpdateType updateType, List createChanges, TableMetadata current, @@ -130,6 +165,7 @@ enum UpdateType { this.readHeaders = readHeaders; this.mutationHeaders = mutationHeaders; this.io = io; + this.keyManagementClient = keyManagementClient; this.updateType = updateType; this.createChanges = createChanges; this.replaceBase = current; @@ -139,6 +175,10 @@ enum UpdateType { this.current = current; } this.endpoints = endpoints; + + // N.B. We don't use this.current due to it being null for the CREATE update type; we still + // want encryption configured for this case. + encryptionPropsFromMetadata(current); } @Override @@ -156,6 +196,19 @@ public TableMetadata refresh() { @Override public void commit(TableMetadata base, TableMetadata metadata) { Endpoint.check(endpoints, Endpoint.V1_UPDATE_TABLE); + + EncryptionManager encryption = encryption(); + if (encryption instanceof StandardEncryptionManager) { + // Add encryption keys to the to-be-committed metadata + TableMetadata.Builder builder = TableMetadata.buildFrom(metadata); + EncryptionUtil.encryptionKeys(encryption).values().forEach(builder::addEncryptionKey); + commitInternal(base, builder.build()); + } else { + commitInternal(base, metadata); + } + } + + private void commitInternal(TableMetadata base, TableMetadata metadata) { Consumer errorHandler; List requirements; List updates; @@ -196,6 +249,21 @@ public void commit(TableMetadata base, TableMetadata metadata) { String.format("Update type %s is not supported", updateType)); } + if (base != null) { + boolean encryptionKeyRemoved = + base.properties().containsKey(TableProperties.ENCRYPTION_TABLE_KEY) + && !metadata.properties().containsKey(TableProperties.ENCRYPTION_TABLE_KEY); + Preconditions.checkArgument( + !encryptionKeyRemoved, "Cannot remove key ID from an encrypted table"); + + boolean encryptionKeyUnchanged = + Objects.equals( + base.properties().get(TableProperties.ENCRYPTION_TABLE_KEY), + metadata.properties().get(TableProperties.ENCRYPTION_TABLE_KEY)); + Preconditions.checkArgument( + encryptionKeyUnchanged, "Cannot modify key ID of an encrypted table"); + } + UpdateTableRequest request = new UpdateTableRequest(requirements, updates); // the error handler will throw necessary exceptions like CommitFailedException and @@ -243,9 +311,47 @@ private boolean reconcileOnSimpleUpdate( } } + // TODO: Integrate and test encryption with REST scan planning @Override public FileIO io() { - return io; + if (tableKeyId == null) { + return io; + } + + if (encryptingFileIO == null) { + encryptingFileIO = EncryptingFileIO.combine(io, encryption()); + } + + return encryptingFileIO; + } + + @Override + public EncryptionManager encryption() { + if (encryptionManager != null) { + return encryptionManager; + } + + if (tableKeyId != null) { + Preconditions.checkArgument( + keyManagementClient != null, + "Cannot create encryption manager without a key management client. Consider setting the '%s' catalog property", + CatalogProperties.ENCRYPTION_KMS_IMPL); + + Map encryptionProperties = + ImmutableMap.of( + TableProperties.ENCRYPTION_TABLE_KEY, + tableKeyId, + TableProperties.ENCRYPTION_DEK_LENGTH, + String.valueOf(encryptionDekLength)); + + encryptionManager = + EncryptionUtil.createEncryptionManager( + encryptedKeys, encryptionProperties, keyManagementClient); + } else { + return PlaintextEncryptionManager.instance(); + } + + return encryptionManager; } private static Long expectedSnapshotIdIfSnapshotAddOnly(List updates) { @@ -285,6 +391,45 @@ private static Long expectedSnapshotIdIfSnapshotAddOnly(List upd return addedSnapshotId; } + private void encryptionPropsFromMetadata(TableMetadata metadata) { + if (metadata == null || metadata.properties() == null) { + return; + } + + // Refresh encryption-related properties and keys on new/refreshed metadata + Map tableProperties = metadata.properties(); + tableKeyId = tableProperties.get(TableProperties.ENCRYPTION_TABLE_KEY); + + if (tableKeyId != null) { + encryptionDekLength = + PropertyUtil.propertyAsInt( + tableProperties, + TableProperties.ENCRYPTION_DEK_LENGTH, + TableProperties.ENCRYPTION_DEK_LENGTH_DEFAULT); + + encryptedKeys = + Optional.ofNullable(metadata.encryptionKeys()) + .map(Lists::newLinkedList) + .orElseGet(Lists::newLinkedList); + + // Include pending encryption keys from the encryption manager + if (encryptionManager != null) { + Set keyIdsFromMetadata = + encryptedKeys.stream().map(EncryptedKey::keyId).collect(Collectors.toSet()); + + for (EncryptedKey keyFromEM : EncryptionUtil.encryptionKeys(encryptionManager).values()) { + if (!keyIdsFromMetadata.contains(keyFromEM.keyId())) { + encryptedKeys.add(keyFromEM); + } + } + } + } + + // Force re-creation of encryption manager + encryptingFileIO = null; + encryptionManager = null; + } + private TableMetadata updateCurrentMetadata(LoadTableResponse response) { // LoadTableResponse is used to deserialize the response, but config is not allowed by the REST // spec so it can be @@ -292,6 +437,7 @@ private TableMetadata updateCurrentMetadata(LoadTableResponse response) { if (current == null || !Objects.equals(current.metadataFileLocation(), response.metadataLocation())) { this.current = checkUUID(current, response.tableMetadata()); + encryptionPropsFromMetadata(current); } return current; diff --git a/core/src/test/java/org/apache/iceberg/rest/TestFreshnessAwareLoading.java b/core/src/test/java/org/apache/iceberg/rest/TestFreshnessAwareLoading.java index a4bb170d1411..eb6edfdc7723 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestFreshnessAwareLoading.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestFreshnessAwareLoading.java @@ -48,6 +48,7 @@ import org.apache.iceberg.TableMetadata; import org.apache.iceberg.catalog.SessionCatalog; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.encryption.KeyManagementClient; import org.apache.iceberg.exceptions.NoSuchTableException; import org.apache.iceberg.exceptions.RESTException; import org.apache.iceberg.io.FileIO; @@ -748,9 +749,10 @@ class CustomTableOps extends RESTTableOperations { Supplier> readHeaders, Supplier> mutationHeaders, FileIO io, + KeyManagementClient kmsClient, TableMetadata current, Set endpoints) { - super(client, path, readHeaders, mutationHeaders, io, current, endpoints); + super(client, path, readHeaders, mutationHeaders, io, kmsClient, current, endpoints); } } @@ -768,10 +770,18 @@ protected RESTTableOperations newTableOps( Supplier> readHeaders, Supplier> mutationHeaders, FileIO fileIO, + KeyManagementClient kmsClient, TableMetadata current, Set supportedEndpoints) { return new CustomTableOps( - restClient, path, readHeaders, mutationHeaders, fileIO, current, supportedEndpoints); + restClient, + path, + readHeaders, + mutationHeaders, + fileIO, + kmsClient, + current, + supportedEndpoints); } } diff --git a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java index e4fa156059d8..77d718b38dd5 100644 --- a/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java +++ b/core/src/test/java/org/apache/iceberg/rest/TestRESTCatalog.java @@ -78,6 +78,7 @@ import org.apache.iceberg.catalog.SessionCatalog; import org.apache.iceberg.catalog.TableCommit; import org.apache.iceberg.catalog.TableIdentifier; +import org.apache.iceberg.encryption.KeyManagementClient; import org.apache.iceberg.exceptions.AlreadyExistsException; import org.apache.iceberg.exceptions.CommitFailedException; import org.apache.iceberg.exceptions.CommitStateUnknownException; @@ -3124,6 +3125,7 @@ public void testCommitStateUnknownNotReconciled() { } @Test + @SuppressWarnings("MethodLength") public void testCustomTableOperationsInjection() throws IOException { AtomicBoolean customTableOpsCalled = new AtomicBoolean(); AtomicBoolean customTransactionTableOpsCalled = new AtomicBoolean(); @@ -3139,9 +3141,17 @@ class CustomRESTTableOperations extends RESTTableOperations { String path, Supplier> headers, FileIO fileIO, + KeyManagementClient keyManagementClient, TableMetadata current, Set supportedEndpoints) { - super(client, path, () -> customHeaders, fileIO, current, supportedEndpoints); + super( + client, + path, + () -> customHeaders, + fileIO, + keyManagementClient, + current, + supportedEndpoints); customTableOpsCalled.set(true); } @@ -3150,6 +3160,7 @@ class CustomRESTTableOperations extends RESTTableOperations { String path, Supplier> headers, FileIO fileIO, + KeyManagementClient keyManagementClient, RESTTableOperations.UpdateType updateType, List createChanges, TableMetadata current, @@ -3159,6 +3170,7 @@ class CustomRESTTableOperations extends RESTTableOperations { path, () -> customHeaders, fileIO, + keyManagementClient, updateType, createChanges, current, @@ -3182,11 +3194,12 @@ protected RESTTableOperations newTableOps( Supplier> readHeaders, Supplier> mutationHeaders, FileIO fileIO, + KeyManagementClient kmsClient, TableMetadata current, Set supportedEndpoints) { RESTTableOperations ops = new CustomRESTTableOperations( - restClient, path, mutationHeaders, fileIO, current, supportedEndpoints); + restClient, path, mutationHeaders, fileIO, kmsClient, current, supportedEndpoints); RESTTableOperations spy = Mockito.spy(ops); capturedOps.set(spy); return spy; @@ -3199,6 +3212,7 @@ protected RESTTableOperations newTableOps( Supplier> readHeaders, Supplier> mutationHeaders, FileIO fileIO, + KeyManagementClient kmsClient, RESTTableOperations.UpdateType updateType, List createChanges, TableMetadata current, @@ -3209,6 +3223,7 @@ protected RESTTableOperations newTableOps( path, mutationHeaders, fileIO, + kmsClient, updateType, createChanges, current, diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java index 3dee6e1e1d54..aebc4a4ca416 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestCTASEncryption.java @@ -32,6 +32,7 @@ import org.apache.iceberg.encryption.UnitestKMS; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Streams; import org.apache.iceberg.spark.CatalogTestBase; @@ -57,6 +58,15 @@ protected static Object[][] parameters() { SparkCatalogConfig.HIVE.catalogName(), SparkCatalogConfig.HIVE.implementation(), appendCatalogEncryptionProperties(SparkCatalogConfig.HIVE.properties()) + }, + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + appendCatalogEncryptionProperties( + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + .build()) } }; } diff --git a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java index 3b36b7bb0a25..777e42545e35 100644 --- a/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java +++ b/spark/v4.1/spark/src/test/java/org/apache/iceberg/spark/sql/TestTableEncryption.java @@ -22,6 +22,7 @@ import static org.apache.iceberg.types.Types.NestedField.optional; import static org.assertj.core.api.Assertions.assertThat; import static org.assertj.core.api.Assertions.assertThatThrownBy; +import static org.assertj.core.api.Assumptions.assumeThat; import java.io.File; import java.io.IOException; @@ -50,8 +51,10 @@ import org.apache.iceberg.io.SeekableInputStream; import org.apache.iceberg.parquet.Parquet; import org.apache.iceberg.relocated.com.google.common.collect.ImmutableList; +import org.apache.iceberg.relocated.com.google.common.collect.ImmutableMap; import org.apache.iceberg.relocated.com.google.common.collect.Maps; import org.apache.iceberg.relocated.com.google.common.collect.Streams; +import org.apache.iceberg.rest.RESTCatalog; import org.apache.iceberg.spark.CatalogTestBase; import org.apache.iceberg.spark.SparkCatalogConfig; import org.apache.iceberg.types.Types; @@ -77,6 +80,15 @@ protected static Object[][] parameters() { SparkCatalogConfig.HIVE.catalogName(), SparkCatalogConfig.HIVE.implementation(), appendCatalogEncryptionProperties(SparkCatalogConfig.HIVE.properties()) + }, + { + SparkCatalogConfig.REST.catalogName(), + SparkCatalogConfig.REST.implementation(), + appendCatalogEncryptionProperties( + ImmutableMap.builder() + .putAll(SparkCatalogConfig.REST.properties()) + .put(CatalogProperties.URI, restCatalog.properties().get(CatalogProperties.URI)) + .build()) } }; } @@ -162,7 +174,6 @@ public void testConcurrentAppendTransactions() { assertThat(currentDataFiles(table)).hasSize(dataFiles.size() + 2); } - // See CatalogTests#testConcurrentReplaceTransactions @TestTemplate public void testConcurrentReplaceTransactions() { validationCatalog.initialize(catalogName, catalogConfig); @@ -171,7 +182,7 @@ public void testConcurrentReplaceTransactions() { DataFile file = currentDataFiles(table).get(0); Schema schema = table.schema(); - // Write data for a replace transaction that will be committed later + // Begin a replace transaction that will be committed second Transaction secondReplace = validationCatalog .buildTable(tableIdent, schema) @@ -185,12 +196,15 @@ public void testConcurrentReplaceTransactions() { .buildTable(tableIdent, schema) .withProperty("encryption.key-id", UnitestKMS.MASTER_KEY_NAME1) .replaceTransaction(); - firstReplace.newFastAppend().appendFile(file).commit(); firstReplace.commitTransaction(); + // This second replace transaction fails but then retries after refreshing latest metadata. secondReplace.commitTransaction(); Table afterSecondReplace = validationCatalog.loadTable(tableIdent); + + // This tests that encryption keys are maintained on refreshing different metadata - if + // they are not, the table will be unreadable and this will fail. assertThat(currentDataFiles(afterSecondReplace)).hasSize(1); } @@ -224,10 +238,14 @@ public void testInsertAndDelete() { @TestTemplate public void testMetadataTamperproofing() throws IOException { + assumeThat(validationCatalog) + .as("RESTCatalog does not store metadata file hashes") + .isNotInstanceOf(RESTCatalog.class); + ChecksumFileSystem fs = ((ChecksumFileSystem) FileSystem.newInstance(new Configuration())); - catalog.initialize(catalogName, catalogConfig); + validationCatalog.initialize(catalogName, catalogConfig); - Table table = catalog.loadTable(tableIdent); + Table table = validationCatalog.loadTable(tableIdent); TableMetadata currentMetadata = ((HasTableOperations) table).operations().current(); Path metadataFile = new Path(currentMetadata.metadataFileLocation()); Path previousMetadataFile = new Path(Iterables.firstOf(currentMetadata.previousFiles()).file()); @@ -238,7 +256,7 @@ public void testMetadataTamperproofing() throws IOException { fs.delete(metadataFile, false); fs.rename(previousMetadataFile, metadataFile); - assertThatThrownBy(() -> catalog.loadTable(tableIdent)) + assertThatThrownBy(() -> validationCatalog.loadTable(tableIdent)) .hasMessageContaining( String.format( "The current metadata file %s might have been modified. Hash of metadata loaded from storage differs from HMS-stored metadata hash.",