Skip to content

Commit

Permalink
Fix storage table data clean up while dropping iceberg materialized view
Browse files Browse the repository at this point in the history
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.
  • Loading branch information
anusudarsan committed Dec 6, 2024
1 parent 999d8a3 commit bed3be2
Show file tree
Hide file tree
Showing 3 changed files with 144 additions and 5 deletions.
Original file line number Diff line number Diff line change
Expand Up @@ -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);
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -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;
Expand All @@ -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;
Expand All @@ -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;
Expand All @@ -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;

Expand All @@ -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 = getBucketName();
dataLake = closer.register(new HiveMinioDataLake(bucketName, HIVE3_IMAGE));
dataLake.start();
}
Expand All @@ -87,6 +109,11 @@ public void tearDown()
closer.close();
}

protected String getBucketName()
{
return "test-hive-catalog-with-hms-" + randomNameSuffix();
}

@Override
protected TrinoCatalog createTrinoCatalog(boolean useUniqueTableLocations)
{
Expand All @@ -111,6 +138,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,
Expand All @@ -135,13 +164,84 @@ public ThriftMetastore createMetastore(Optional<ConnectorIdentity> 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<SchemaTableName> materializedViews = catalog.listTables(SESSION, Optional.of(namespace)).stream()
.filter(info -> info.extendedRelationType() == TableInfo.ExtendedRelationType.TRINO_MATERIALIZED_VIEW)
.map(TableInfo::tableName)
.toList();
assertThat(materializedViews.size()).isEqualTo(1);
assertThat(materializedViews.get(0).getTableName()).isEqualTo(materializedViewName);
Optional<ConnectorMaterializedViewDefinition> materializedView = catalog.getMaterializedView(SESSION, materializedViews.get(0));
assertThat(materializedView).isPresent();
String storageTableName;
if (isHideMaterializedViewStorageTable()) {
storageTableName = materializedViewName;
}
else {
Optional<CatalogSchemaTableName> 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<String, Object> defaultNamespaceProperties(String newNamespaceName)
{
return Map.of(IcebergSchemaProperties.LOCATION_PROPERTY, "s3://%s/%s".formatted(bucketName, newNamespaceName));
return Map.of(IcebergSchemaProperties.LOCATION_PROPERTY, getNamespaceLocation(newNamespaceName));
}

private String getNamespaceLocation(String newNamespaceName)
{
return "s3://%s/%s".formatted(bucketName, newNamespaceName);
}
}
Original file line number Diff line number Diff line change
@@ -0,0 +1,39 @@
/*
* 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 io.trino.testing.TestingNames.randomNameSuffix;
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 TestTrinoLegacyHiveCatalogWithHiveMetastore
extends TestTrinoHiveCatalogWithHiveMetastore
{
@Override
protected String getBucketName()
{
return "test-legacy-hive-catalog-with-hms-" + randomNameSuffix();
}

@Override
protected boolean isHideMaterializedViewStorageTable()
{
return false;
}
}

0 comments on commit bed3be2

Please sign in to comment.