From c7a2d54c28787c4c44dce848e2e1602972e2100f Mon Sep 17 00:00:00 2001 From: Anu Sudarsan Date: Thu, 5 Dec 2024 17:51:23 -0500 Subject: [PATCH] Fix storage table data clean up while dropping iceberg materialized view This is for cleaning up data files in legacy mode i.e iceberg.materialized-views.hide-storage-table is set to false. Delegating the drop table to metastore does not clean up the data files since for HMS, the iceberg table is registered as an "external" table. So to fix this instead of delegating to metastore, have the connector do the drop of the table and data files associated with it. --- .../iceberg/catalog/hms/TrinoHiveCatalog.java | 2 +- ...TestTrinoHiveCatalogWithHiveMetastore.java | 105 +++++++++++++++++- ...HiveCatalogWithoutHiddenStorageTables.java | 32 ++++++ 3 files changed, 133 insertions(+), 6 deletions(-) create mode 100644 plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithoutHiddenStorageTables.java 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 8e6b6289dd94..b450063dba6f 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 @@ -715,7 +715,7 @@ private void dropMaterializedViewStorage(ConnectorSession session, io.trino.meta String storageSchema = Optional.ofNullable(view.getParameters().get(STORAGE_SCHEMA)) .orElse(viewName.getSchemaName()); try { - metastore.dropTable(storageSchema, storageTableName, true); + dropTable(session, new SchemaTableName(storageSchema, storageTableName)); } catch (TrinoException e) { log.warn(e, "Failed to drop storage table '%s.%s' for materialized view '%s'", storageSchema, storageTableName, viewName); 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 208fd337fc66..587f63d2accf 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 @@ -13,8 +13,13 @@ */ package io.trino.plugin.iceberg.catalog.hms; +import com.google.common.collect.ImmutableList; +import com.google.common.collect.ImmutableMap; import com.google.common.collect.ImmutableSet; +import io.airlift.log.Logger; import io.airlift.units.Duration; +import io.trino.filesystem.Location; +import io.trino.filesystem.TrinoFileSystem; import io.trino.filesystem.TrinoFileSystemFactory; import io.trino.filesystem.hdfs.HdfsFileSystemFactory; import io.trino.hdfs.DynamicHdfsConfiguration; @@ -25,6 +30,7 @@ import io.trino.hdfs.authentication.NoHdfsAuthentication; import io.trino.hdfs.s3.HiveS3Config; import io.trino.hdfs.s3.TrinoS3ConfigurationInitializer; +import io.trino.metastore.TableInfo; import io.trino.plugin.base.util.AutoCloseableCloser; import io.trino.plugin.hive.TrinoViewHiveMetastore; import io.trino.plugin.hive.containers.HiveMinioDataLake; @@ -33,18 +39,25 @@ import io.trino.plugin.hive.metastore.thrift.ThriftMetastore; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreConfig; import io.trino.plugin.hive.metastore.thrift.ThriftMetastoreFactory; -import io.trino.plugin.iceberg.IcebergConfig; import io.trino.plugin.iceberg.IcebergSchemaProperties; import io.trino.plugin.iceberg.catalog.BaseTrinoCatalogTest; import io.trino.plugin.iceberg.catalog.TrinoCatalog; import io.trino.spi.catalog.CatalogName; +import io.trino.spi.connector.CatalogSchemaTableName; +import io.trino.spi.connector.ConnectorMaterializedViewDefinition; +import io.trino.spi.connector.SchemaTableName; import io.trino.spi.security.ConnectorIdentity; +import io.trino.spi.security.PrincipalType; +import io.trino.spi.security.TrinoPrincipal; import io.trino.spi.type.TestingTypeManager; import org.junit.jupiter.api.AfterAll; import org.junit.jupiter.api.BeforeAll; +import org.junit.jupiter.api.Test; import org.junit.jupiter.api.TestInstance; import org.junit.jupiter.api.parallel.Execution; +import java.io.IOException; +import java.util.List; import java.util.Map; import java.util.Optional; import java.util.Set; @@ -54,10 +67,16 @@ import static io.trino.plugin.hive.TestingThriftHiveMetastoreBuilder.testingThriftHiveMetastoreBuilder; import static io.trino.plugin.hive.containers.HiveHadoop.HIVE3_IMAGE; import static io.trino.plugin.hive.metastore.cache.CachingHiveMetastore.createPerTransactionCache; +import static io.trino.plugin.iceberg.IcebergFileFormat.PARQUET; +import static io.trino.plugin.iceberg.IcebergTableProperties.FILE_FORMAT_PROPERTY; +import static io.trino.plugin.iceberg.IcebergTableProperties.FORMAT_VERSION_PROPERTY; +import static io.trino.spi.type.IntegerType.INTEGER; +import static io.trino.testing.TestingConnectorSession.SESSION; import static io.trino.testing.TestingNames.randomNameSuffix; import static io.trino.testing.containers.Minio.MINIO_ACCESS_KEY; import static io.trino.testing.containers.Minio.MINIO_SECRET_KEY; import static java.util.concurrent.TimeUnit.MINUTES; +import static org.assertj.core.api.Assertions.assertThat; import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; @@ -66,15 +85,18 @@ public class TestTrinoHiveCatalogWithHiveMetastore extends BaseTrinoCatalogTest { - private static final String bucketName = "test-hive-catalog-with-hms-" + randomNameSuffix(); + private static final Logger LOG = Logger.get(TestTrinoHiveCatalogWithHiveMetastore.class); private AutoCloseableCloser closer = AutoCloseableCloser.create(); // Use MinIO for storage, since HDFS is hard to get working in a unit test private HiveMinioDataLake dataLake; + private TrinoFileSystem fileSystem; + private String bucketName; @BeforeAll public void setUp() { + bucketName = "test-hive-catalog-with-hms-" + randomNameSuffix(); dataLake = closer.register(new HiveMinioDataLake(bucketName, HIVE3_IMAGE)); dataLake.start(); } @@ -111,6 +133,8 @@ protected TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations) .metastoreClient(dataLake.getHiveHadoop().getHiveMetastoreEndpoint()) .build(closer::register); CachingHiveMetastore metastore = createPerTransactionCache(new BridgingHiveMetastore(thriftMetastore), 1000); + fileSystem = fileSystemFactory.create(SESSION); + return new TrinoHiveCatalog( new CatalogName("catalog"), metastore, @@ -135,13 +159,84 @@ public ThriftMetastore createMetastore(Optional identity) useUniqueTableLocations, false, false, - new IcebergConfig().isHideMaterializedViewStorageTable(), + isHideMaterializedViewStorageTable(), directExecutor()); } + protected boolean isHideMaterializedViewStorageTable() + { + return true; + } + + @Test + public void testCreateMaterializedView() + throws IOException + { + TrinoCatalog catalog = createTrinoCatalog(false); + String namespace = "test_create_mv_" + randomNameSuffix(); + String materializedViewName = "materialized_view_name"; + try { + catalog.createNamespace(SESSION, namespace, defaultNamespaceProperties(namespace), new TrinoPrincipal(PrincipalType.USER, SESSION.getUser())); + catalog.createMaterializedView( + SESSION, + new SchemaTableName(namespace, materializedViewName), + new ConnectorMaterializedViewDefinition( + "SELECT * FROM tpch.tiny.nation", + Optional.empty(), + Optional.of("catalog_name"), + Optional.of("schema_name"), + ImmutableList.of(new ConnectorMaterializedViewDefinition.Column("col1", INTEGER.getTypeId(), Optional.empty())), + Optional.empty(), + Optional.empty(), + Optional.empty(), + ImmutableList.of()), + ImmutableMap.of(FILE_FORMAT_PROPERTY, PARQUET, FORMAT_VERSION_PROPERTY, 1), + false, + false); + List materializedViews = catalog.listTables(SESSION, Optional.of(namespace)).stream() + .filter(info -> info.extendedRelationType() == TableInfo.ExtendedRelationType.TRINO_MATERIALIZED_VIEW) + .map(TableInfo::tableName) + .toList(); + assertThat(materializedViews).hasSize(1); + assertThat(materializedViews.getFirst().getTableName()).isEqualTo(materializedViewName); + Optional materializedView = catalog.getMaterializedView(SESSION, materializedViews.getFirst()); + assertThat(materializedView).isPresent(); + String storageTableName; + if (isHideMaterializedViewStorageTable()) { + storageTableName = materializedViewName; + } + else { + Optional storageTable = materializedView.get().getStorageTable(); + assertThat(storageTable).isPresent(); + storageTableName = storageTable.get().getSchemaTableName().getTableName(); + } + Location dataLocation = Location.of(getNamespaceLocation(namespace) + "/" + storageTableName); + assertThat(fileSystem.newInputFile(dataLocation).exists()) + .describedAs("The directory corresponding to the table data for materialized view must exist") + .isTrue(); + catalog.dropMaterializedView(SESSION, new SchemaTableName(namespace, materializedViewName)); + assertThat(fileSystem.newInputFile(dataLocation).exists()) + .describedAs("The materialized view drop should also delete the data files associated with it") + .isFalse(); + } + finally { + try { + catalog.dropNamespace(SESSION, namespace); + } + catch (Exception e) { + LOG.warn("Failed to clean up namespace: %s", namespace); + } + } + } + @Override - protected Map defaultNamespaceProperties(String newNamespaceName) + protected Map defaultNamespaceProperties(String namespaceName) + { + return Map.of(IcebergSchemaProperties.LOCATION_PROPERTY, getNamespaceLocation(namespaceName)); + } + + private String getNamespaceLocation(String namespaceName) { - return Map.of(IcebergSchemaProperties.LOCATION_PROPERTY, "s3://%s/%s".formatted(bucketName, newNamespaceName)); + return "s3://%s/%s".formatted(bucketName, namespaceName); } } diff --git a/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithoutHiddenStorageTables.java b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithoutHiddenStorageTables.java new file mode 100644 index 000000000000..7e904c1271c7 --- /dev/null +++ b/plugin/trino-iceberg/src/test/java/io/trino/plugin/iceberg/catalog/hms/TestTrinoHiveCatalogWithoutHiddenStorageTables.java @@ -0,0 +1,32 @@ +/* + * 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.catalog.hms; + +import org.junit.jupiter.api.TestInstance; +import org.junit.jupiter.api.parallel.Execution; + +import static org.junit.jupiter.api.TestInstance.Lifecycle.PER_CLASS; +import static org.junit.jupiter.api.parallel.ExecutionMode.CONCURRENT; + +@TestInstance(PER_CLASS) +@Execution(CONCURRENT) +public class TestTrinoHiveCatalogWithoutHiddenStorageTables + extends TestTrinoHiveCatalogWithHiveMetastore +{ + @Override + protected boolean isHideMaterializedViewStorageTable() + { + return false; + } +}